Repository: spark Updated Branches: refs/heads/branch-1.0 d0dde920d -> a4c3a8069
[SPARK-2187] Explain should not run the optimizer twice. @yhuai @marmbrus @concretevitamin Author: Reynold Xin <[email protected]> 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. (cherry picked from commit 640c294369f49a7602c33c7c389088aec8a316d3) Signed-off-by: Reynold Xin <[email protected]> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a4c3a806 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a4c3a806 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a4c3a806 Branch: refs/heads/branch-1.0 Commit: a4c3a806906dffa0ee7a2c54a3b3f5cfb3225639 Parents: d0dde92 Author: Reynold Xin <[email protected]> Authored: Wed Jun 18 22:44:12 2014 -0700 Committer: Reynold Xin <[email protected]> Committed: Wed Jun 18 22:44:21 2014 -0700 ---------------------------------------------------------------------- .../spark/sql/execution/SparkStrategies.scala | 5 ++--- .../org/apache/spark/sql/execution/commands.scala | 16 ++++++++++++---- .../spark/sql/hive/execution/HiveQuerySuite.scala | 5 +---- 3 files changed, 15 insertions(+), 11 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/spark/blob/a4c3a806/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 70c1171..feb280d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -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 http://git-wip-us.apache.org/repos/asf/spark/blob/a4c3a806/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 39b3246..f5d0834 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -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 { /** @@ -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))) http://git-wip-us.apache.org/repos/asf/spark/blob/a4c3a806/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala ---------------------------------------------------------------------- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index fe698f0..8b2bdd5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -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") {
