@@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow
24
24
import org .apache .spark .sql .catalyst .catalog .{HiveTableRelation , SessionCatalog }
25
25
import org .apache .spark .sql .catalyst .expressions ._
26
26
import org .apache .spark .sql .catalyst .expressions .aggregate ._
27
+ import org .apache .spark .sql .catalyst .planning .PhysicalOperation
27
28
import org .apache .spark .sql .catalyst .plans .logical ._
28
29
import org .apache .spark .sql .catalyst .rules .Rule
29
30
import org .apache .spark .sql .catalyst .util .{CaseInsensitiveMap , DateTimeUtils }
@@ -49,9 +50,13 @@ case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[Logic
49
50
}
50
51
51
52
plan.transform {
52
- case a @ Aggregate (_, aggExprs, child @ PartitionedRelation (_, attrs, filters, rel)) =>
53
+ case a @ Aggregate (_, aggExprs, child @ PhysicalOperation (
54
+ projectList, filters, PartitionedRelation (partAttrs, rel))) =>
53
55
// We only apply this optimization when only partitioned attributes are scanned.
54
- if (a.references.subsetOf(attrs)) {
56
+ if (AttributeSet ((projectList ++ filters).flatMap(_.references)).subsetOf(partAttrs)) {
57
+ // The project list and filters all only refer to partition attributes, which means the
58
+ // the Aggregator operator can also only refer to partition attributes, and filters are
59
+ // all partition filters. This is a metadata only query we can optimize.
55
60
val aggFunctions = aggExprs.flatMap(_.collect {
56
61
case agg : AggregateExpression => agg
57
62
})
@@ -102,7 +107,7 @@ case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[Logic
102
107
partFilters : Seq [Expression ]): LogicalPlan = {
103
108
// this logic comes from PruneFileSourcePartitions. it ensures that the filter names match the
104
109
// relation's schema. PartitionedRelation ensures that the filters only reference partition cols
105
- val relFilters = partFilters.map { e =>
110
+ val normalizedFilters = partFilters.map { e =>
106
111
e transform {
107
112
case a : AttributeReference =>
108
113
a.withName(relation.output.find(_.semanticEquals(a)).get.name)
@@ -114,11 +119,8 @@ case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[Logic
114
119
relation match {
115
120
case l @ LogicalRelation (fsRelation : HadoopFsRelation , _, _, isStreaming) =>
116
121
val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l)
117
- val partitionData = fsRelation.location.listFiles(relFilters, Nil )
118
- // partition data may be a stream, which can cause serialization to hit stack level too
119
- // deep exceptions because it is a recursive structure in memory. converting to array
120
- // avoids the problem.
121
- LocalRelation (partAttrs, partitionData.map(_.values).toArray, isStreaming)
122
+ val partitionData = fsRelation.location.listFiles(normalizedFilters, Nil )
123
+ LocalRelation (partAttrs, partitionData.map(_.values), isStreaming)
122
124
123
125
case relation : HiveTableRelation =>
124
126
val partAttrs = getPartitionAttrs(relation.tableMeta.partitionColumnNames, relation)
@@ -127,7 +129,7 @@ case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[Logic
127
129
val timeZoneId = caseInsensitiveProperties.get(DateTimeUtils .TIMEZONE_OPTION )
128
130
.getOrElse(SQLConf .get.sessionLocalTimeZone)
129
131
val partitions = if (partFilters.nonEmpty) {
130
- catalog.listPartitionsByFilter(relation.tableMeta.identifier, relFilters )
132
+ catalog.listPartitionsByFilter(relation.tableMeta.identifier, normalizedFilters )
131
133
} else {
132
134
catalog.listPartitions(relation.tableMeta.identifier)
133
135
}
@@ -137,10 +139,7 @@ case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[Logic
137
139
Cast (Literal (p.spec(attr.name)), attr.dataType, Option (timeZoneId)).eval()
138
140
})
139
141
}
140
- // partition data may be a stream, which can cause serialization to hit stack level too
141
- // deep exceptions because it is a recursive structure in memory. converting to array
142
- // avoids the problem.
143
- LocalRelation (partAttrs, partitionData.toArray)
142
+ LocalRelation (partAttrs, partitionData)
144
143
145
144
case _ =>
146
145
throw new IllegalStateException (s " unrecognized table scan node: $relation, " +
@@ -151,44 +150,21 @@ case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[Logic
151
150
152
151
/**
153
152
* A pattern that finds the partitioned table relation node inside the given plan, and returns a
154
- * pair of the partition attributes, partition filters, and the table relation node.
155
- *
156
- * It keeps traversing down the given plan tree if there is a [[Project ]] or [[Filter ]] with
157
- * deterministic expressions, and returns result after reaching the partitioned table relation
158
- * node.
153
+ * pair of the partition attributes and the table relation node.
159
154
*/
160
155
object PartitionedRelation extends PredicateHelper {
161
156
162
- def unapply (
163
- plan : LogicalPlan ): Option [(AttributeSet , AttributeSet , Seq [Expression ], LogicalPlan )] = {
157
+ def unapply (plan : LogicalPlan ): Option [(AttributeSet , LogicalPlan )] = {
164
158
plan match {
165
159
case l @ LogicalRelation (fsRelation : HadoopFsRelation , _, _, _)
166
- if fsRelation.partitionSchema.nonEmpty =>
160
+ if fsRelation.partitionSchema.nonEmpty =>
167
161
val partAttrs = AttributeSet (getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l))
168
- Some ((partAttrs, partAttrs, Nil , l))
162
+ Some ((partAttrs, l))
169
163
170
164
case relation : HiveTableRelation if relation.tableMeta.partitionColumnNames.nonEmpty =>
171
165
val partAttrs = AttributeSet (
172
166
getPartitionAttrs(relation.tableMeta.partitionColumnNames, relation))
173
- Some ((partAttrs, partAttrs, Nil , relation))
174
-
175
- case p @ Project (projectList, child) if projectList.forall(_.deterministic) =>
176
- unapply(child).flatMap { case (partAttrs, attrs, filters, relation) =>
177
- if (p.references.subsetOf(attrs)) {
178
- Some ((partAttrs, p.outputSet, filters, relation))
179
- } else {
180
- None
181
- }
182
- }
183
-
184
- case f @ Filter (condition, child) if condition.deterministic =>
185
- unapply(child).flatMap { case (partAttrs, attrs, filters, relation) =>
186
- if (f.references.subsetOf(partAttrs)) {
187
- Some ((partAttrs, attrs, splitConjunctivePredicates(condition) ++ filters, relation))
188
- } else {
189
- None
190
- }
191
- }
167
+ Some ((partAttrs, relation))
192
168
193
169
case _ => None
194
170
}
0 commit comments