17
17
18
18
package org .apache .spark .sql .catalyst .analysis
19
19
20
- import scala .collection .mutable .ArrayBuffer
21
-
22
20
import org .apache .spark .sql .AnalysisException
23
21
import org .apache .spark .sql .catalyst .{SimpleCatalystConf , CatalystConf }
24
22
import org .apache .spark .sql .catalyst .expressions ._
@@ -153,9 +151,11 @@ class Analyzer(
153
151
}
154
152
155
153
def apply (plan : LogicalPlan ): LogicalPlan = plan transformUp {
156
- case Aggregate (groups, aggs, child) if child.resolved =>
154
+ case Aggregate (groups, aggs, child)
155
+ if child.resolved && aggs.exists(_.isInstanceOf [UnresolvedAlias ]) =>
157
156
Aggregate (groups, assignAliases(aggs), child)
158
- case Project (projectList, child) if child.resolved =>
157
+ case Project (projectList, child)
158
+ if child.resolved && projectList.exists(_.isInstanceOf [UnresolvedAlias ]) =>
159
159
Project (assignAliases(projectList), child)
160
160
}
161
161
}
@@ -371,12 +371,10 @@ class Analyzer(
371
371
q.asInstanceOf [GroupingAnalytics ].gid
372
372
case u @ UnresolvedAttribute (nameParts) =>
373
373
// Leave unchanged if resolution fails. Hopefully will be resolved next round.
374
- val result = withPosition(u) {
375
- q.resolveChildren(nameParts, resolver).map {
376
- case UnresolvedAlias (child) => child
377
- case other => other
378
- }.getOrElse(u)
379
- }
374
+ val result =
375
+ withPosition(u) {
376
+ q.resolveChildren(nameParts, resolver).map(trimUnresolvedAlias).getOrElse(u)
377
+ }
380
378
logDebug(s " Resolving $u to $result" )
381
379
result
382
380
case UnresolvedExtractValue (child, fieldExpr) if child.resolved =>
@@ -402,6 +400,11 @@ class Analyzer(
402
400
exprs.exists(_.collect { case _ : Star => true }.nonEmpty)
403
401
}
404
402
403
+ private def trimUnresolvedAlias (ne : NamedExpression ) = ne match {
404
+ case UnresolvedAlias (child) => child
405
+ case other => other
406
+ }
407
+
405
408
private def resolveSortOrders (ordering : Seq [SortOrder ], plan : LogicalPlan , throws : Boolean ) = {
406
409
ordering.map { order =>
407
410
// Resolve SortOrder in one round.
@@ -411,7 +414,7 @@ class Analyzer(
411
414
try {
412
415
val newOrder = order transformUp {
413
416
case u @ UnresolvedAttribute (nameParts) =>
414
- plan.resolve(nameParts, resolver).getOrElse(u)
417
+ plan.resolve(nameParts, resolver).map(trimUnresolvedAlias). getOrElse(u)
415
418
case UnresolvedExtractValue (child, fieldName) if child.resolved =>
416
419
ExtractValue (child, fieldName, resolver)
417
420
}
0 commit comments