Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
feedback for marmbrus
  • Loading branch information
cloud-fan committed Mar 7, 2015
commit 187d97e0b35c351188f60781419e744a8fc5c060
Original file line number Diff line number Diff line change
Expand Up @@ -256,11 +256,22 @@ class Analyzer(catalog: Catalog,

case q: LogicalPlan =>
logTrace(s"Attempting to resolve ${q.simpleString}")
q transformExpressionsUp {
q transformExpressionsUp {
case u @ UnresolvedAttribute(name) if resolver(name, VirtualColumn.groupingIdName) &&
q.isInstanceOf[GroupingAnalytics] =>
// Resolve the virtual column GROUPING__ID for the operator GroupingAnalytics
q.asInstanceOf[GroupingAnalytics].gid
case u @ UnresolvedAttribute(name) if q.isInstanceOf[Sort] =>
val s = q.asInstanceOf[Sort]
val input = s.child match {
case Project(list, c) => list.filter {
case Alias(g: GetField, _) => false
case Alias(g: GetItem, _) => false
case _ => true
}.map(_.toAttribute)
case _ => s.child.flatMap(_.output)
}
s.resolve(name, input, resolver).getOrElse(u)
case u @ UnresolvedAttribute(name) =>
// Leave unchanged if resolution fails. Hopefully will be resolved next round.
val result = q.resolveChildren(name, resolver).getOrElse(u)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging {
}

/** Performs attribute resolution given a name and a sequence of possible attributes. */
protected def resolve(
def resolve(
name: String,
input: Seq[Attribute],
resolver: Resolver): Option[NamedExpression] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark.sql.catalyst.plans.logical

import org.apache.spark.sql.catalyst.analysis.Resolver
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -153,18 +152,6 @@ case class Sort(
global: Boolean,
child: LogicalPlan) extends UnaryNode {
override def output = child.output

override def resolveChildren(name: String, resolver: Resolver) = {
val input = child match {
case Project(list, c) => list.filter {
case Alias(g: GetField, _) => false
case Alias(g: GetItem, _) => false
case _ => true
}.map(_.toAttribute)
case _ => child.flatMap(_.output)
}
resolve(name, input, resolver)
}
}

case class Aggregate(
Expand Down