Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-1704][SQL] Fully support EXPLAIN commands as SchemaRDD. #1003

Closed
wants to merge 6 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
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
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
6 changes: 6 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
val sparkContext = self.sparkContext

val strategies: Seq[Strategy] =
CommandStrategy(self) ::
TakeOrdered ::
PartialAggregation ::
HashJoin ::
Expand Down Expand Up @@ -255,6 +256,11 @@ class SQLContext(@transient val sparkContext: SparkContext)
Batch("Prepare Expressions", Once, new BindReferences[SparkPlan]) :: Nil
}

// TODO: or should we make QueryExecution protected[sql]?
protected[sql] def mkQueryExecution(plan: LogicalPlan) = new QueryExecution {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method already exists above as executePlan.

val logical = plan
}

/**
* The primary workflow for executing relational queries using Spark. Designed to allow easy
* access to the intermediate phases of query execution for developers.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -217,4 +217,15 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case _ => Nil
}
}

// TODO: this should be merged with SPARK-1508's SetCommandStrategy
case class CommandStrategy(context: SQLContext) extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.ExplainCommand(child) =>
val qe = context.mkQueryExecution(child)
Seq(execution.ExplainCommandPhysical(qe.executedPlan, plan.output)(context))
case _ => Nil
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution

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, 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))
}

override def otherCopyArgs = context :: Nil
}
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
val hiveContext = self

override val strategies: Seq[Strategy] = Seq(
CommandStrategy(self),
TakeOrdered,
ParquetOperations,
HiveTableScans,
Expand Down Expand Up @@ -303,7 +304,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
*/
def stringResult(): Seq[String] = analyzed match {
case NativeCommand(cmd) => runSqlHive(cmd)
case ExplainCommand(plan) => new QueryExecution { val logical = plan }.toString.split("\n")
case ExplainCommand(plan) => mkQueryExecution(plan).toString.split("\n")
case query =>
val result: Seq[Seq[Any]] = toRdd.collect().toSeq
// We need the types so we can output struct field names
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.hive.execution

import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive

/**
* A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution.
Expand Down Expand Up @@ -159,4 +160,15 @@ class HiveQuerySuite extends HiveComparisonTest {
hql("SHOW TABLES").toString
hql("SELECT * FROM src").toString
}

test("SPARK-1704: Explain commands as a SchemaRDD") {
hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
val rdd = hql("explain select key, count(value) from src group by key")
assert(rdd.collect().size == 1)
assert(rdd.toString.contains("ExplainCommand"))
assert(rdd.filter(row => row.toString.contains("ExplainCommand")).collect().size == 0,
"actual contents of the result should be the plans of the query to be explained")
TestHive.reset()
}

}