Skip to content

Commit

Permalink
[SPARK-2187] Explain should not run the optimizer twice.
Browse files Browse the repository at this point in the history
@yhuai @marmbrus @concretevitamin

Author: Reynold Xin <rxin@apache.org>

Closes #1123 from rxin/explain and squashes the following commits:

def83b0 [Reynold Xin] Update unit tests for explain.
a9d3ba8 [Reynold Xin] [SPARK-2187] Explain should not run the optimizer twice.
  • Loading branch information
rxin committed Jun 19, 2014
1 parent 566f70f commit 640c294
Show file tree
Hide file tree
Showing 3 changed files with 15 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -250,9 +250,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.SetCommand(key, value) =>
Seq(execution.SetCommand(key, value, plan.output)(context))
case logical.ExplainCommand(child) =>
val sparkPlan = context.executePlan(child).sparkPlan
Seq(execution.ExplainCommand(sparkPlan, plan.output)(context))
case logical.ExplainCommand(logicalPlan) =>
Seq(execution.ExplainCommand(logicalPlan, plan.output)(context))
case logical.CacheCommand(tableName, cache) =>
Seq(execution.CacheCommand(tableName, cache)(context))
case _ => Nil
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{SQLContext, Row}
import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan

trait Command {
/**
Expand Down Expand Up @@ -71,16 +72,23 @@ case class SetCommand(
}

/**
* An explain command for users to see how a command will be executed.
*
* Note that this command takes in a logical plan, runs the optimizer on the logical plan
* (but do NOT actually execute it).
*
* :: DeveloperApi ::
*/
@DeveloperApi
case class ExplainCommand(
child: SparkPlan, output: Seq[Attribute])(
logicalPlan: LogicalPlan, output: Seq[Attribute])(
@transient context: SQLContext)
extends UnaryNode with Command {
extends LeafNode with Command {

// Actually "EXPLAIN" command doesn't cause any side effect.
override protected[sql] lazy val sideEffectResult: Seq[String] = this.toString.split("\n")
// Run through the optimizer to generate the physical plan.
override protected[sql] lazy val sideEffectResult: Seq[String] = {
"Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n")
}

def execute(): RDD[Row] = {
val explanation = sideEffectResult.map(row => new GenericRow(Array[Any](row)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -202,12 +202,9 @@ class HiveQuerySuite extends HiveComparisonTest {
}
}

private val explainCommandClassName =
classOf[execution.ExplainCommand].getSimpleName.stripSuffix("$")

def isExplanation(result: SchemaRDD) = {
val explanation = result.select('plan).collect().map { case Row(plan: String) => plan }
explanation.size > 1 && explanation.head.startsWith(explainCommandClassName)
explanation.size > 1 && explanation.head.startsWith("Physical execution plan")
}

test("SPARK-1704: Explain commands as a SchemaRDD") {
Expand Down

0 comments on commit 640c294

Please sign in to comment.