Skip to content

Commit

Permalink
Modify output().
Browse files Browse the repository at this point in the history
  • Loading branch information
concretevitamin committed Jun 7, 2014
1 parent 719ada9 commit 1bfa379
Show file tree
Hide file tree
Showing 3 changed files with 7 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.types.StructType
import org.apache.spark.sql.catalyst.types.{StringType, StructType}
import org.apache.spark.sql.catalyst.trees

abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
Expand Down Expand Up @@ -102,7 +102,7 @@ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] {
*/
abstract class Command extends LeafNode {
self: Product =>
def output = Seq.empty
def output: Seq[Attribute] = Seq.empty
}

/**
Expand All @@ -115,7 +115,9 @@ case class NativeCommand(cmd: String) extends Command
* Returned by a parser when the users only wants to see what query plan would be executed, without
* actually performing the execution.
*/
case class ExplainCommand(plan: LogicalPlan) extends Command
case class ExplainCommand(plan: LogicalPlan) extends Command {
override def output = Seq(AttributeReference("plan", StringType, nullable = false)())
}

/**
* A logical plan node with single child.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.ExplainCommand(child) =>
val qe = context.mkQueryExecution(child)
Seq(execution.ExplainCommandPhysical(qe.executedPlan)(context))
Seq(execution.ExplainCommandPhysical(qe.executedPlan, plan.output)(context))
case _ => Nil
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,12 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{SQLContext, Row}
import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute}

case class ExplainCommandPhysical(child: SparkPlan)
case class ExplainCommandPhysical(child: SparkPlan, output: Seq[Attribute])
(@transient context: SQLContext) extends UnaryNode {
def execute(): RDD[Row] = {
val planString = new GenericRow(Array[Any](child.toString))
context.sparkContext.parallelize(Seq(planString))
}

def output: Seq[Attribute] = child.output // right thing to do?

override def otherCopyArgs = context :: Nil
}

0 comments on commit 1bfa379

Please sign in to comment.