Skip to content

Commit 4cfd23c

Browse files
committed
fix order by
1 parent d18f401 commit 4cfd23c

File tree

3 files changed

+16
-13
lines changed

3 files changed

+16
-13
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala

Lines changed: 14 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,6 @@
1717

1818
package org.apache.spark.sql.catalyst.analysis
1919

20-
import scala.collection.mutable.ArrayBuffer
21-
2220
import org.apache.spark.sql.AnalysisException
2321
import org.apache.spark.sql.catalyst.{SimpleCatalystConf, CatalystConf}
2422
import org.apache.spark.sql.catalyst.expressions._
@@ -153,9 +151,11 @@ class Analyzer(
153151
}
154152

155153
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]) =>
157156
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]) =>
159159
Project(assignAliases(projectList), child)
160160
}
161161
}
@@ -371,12 +371,10 @@ class Analyzer(
371371
q.asInstanceOf[GroupingAnalytics].gid
372372
case u @ UnresolvedAttribute(nameParts) =>
373373
// 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+
}
380378
logDebug(s"Resolving $u to $result")
381379
result
382380
case UnresolvedExtractValue(child, fieldExpr) if child.resolved =>
@@ -402,6 +400,11 @@ class Analyzer(
402400
exprs.exists(_.collect { case _: Star => true }.nonEmpty)
403401
}
404402

403+
private def trimUnresolvedAlias(ne: NamedExpression) = ne match {
404+
case UnresolvedAlias(child) => child
405+
case other => other
406+
}
407+
405408
private def resolveSortOrders(ordering: Seq[SortOrder], plan: LogicalPlan, throws: Boolean) = {
406409
ordering.map { order =>
407410
// Resolve SortOrder in one round.
@@ -411,7 +414,7 @@ class Analyzer(
411414
try {
412415
val newOrder = order transformUp {
413416
case u @ UnresolvedAttribute(nameParts) =>
414-
plan.resolve(nameParts, resolver).getOrElse(u)
417+
plan.resolve(nameParts, resolver).map(trimUnresolvedAlias).getOrElse(u)
415418
case UnresolvedExtractValue(child, fieldName) if child.resolved =>
416419
ExtractValue(child, fieldName, resolver)
417420
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -222,6 +222,6 @@ case class UnresolvedAlias(child: Expression) extends NamedExpression
222222

223223
override lazy val resolved = false
224224

225-
override def eval(input: Row = null): Any =
225+
override def eval(input: InternalRow = null): Any =
226226
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
227227
}

sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental}
3232
import org.apache.spark.api.java.JavaRDD
3333
import org.apache.spark.api.python.SerDeUtil
3434
import org.apache.spark.rdd.RDD
35-
import org.apache.spark.sql.catalyst.analysis.{MultiAlias, ResolvedStar, UnresolvedAttribute}
35+
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, ResolvedStar, UnresolvedAttribute}
3636
import org.apache.spark.sql.catalyst.expressions._
3737
import org.apache.spark.sql.catalyst.plans.logical.{Filter, _}
3838
import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}

0 commit comments

Comments
 (0)