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

Conversation

concretevitamin
Copy link
Contributor

This PR attempts to resolve SPARK-1704 by introducing a physical plan for EXPLAIN commands, which just prints out the debug string (containing various SparkSQL's plans) of the corresponding QueryExecution for the actual query.

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished. All automated tests passed.

@AmplabJenkins
Copy link

All automated tests passed.
Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15520/

case class ExplainCommandPhysical(child: SparkPlan)
(@transient context: SQLContext) extends UnaryNode {
def execute(): RDD[Row] = {
val lines = child.toString.split("\n").map(s => new GenericRow(Array[Any](s)))
Copy link
Contributor

Choose a reason for hiding this comment

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

seems a little bit strange to have one row per row ... I think the plan should be a single value.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@rxin
Copy link
Contributor

rxin commented Jun 7, 2014

This might not be your problem, but when I tried the following, I got ...

scala> c.hql("explain select key, count(value) from src group by key").collect()
14/06/06 23:58:05 INFO parse.ParseDriver: Parsing command: explain select key, count(value) from src group by key
14/06/06 23:58:05 INFO parse.ParseDriver: Parse Completed
14/06/06 23:58:05 INFO analysis.Analyzer: Max iterations (2) reached for batch MultiInstanceRelations
14/06/06 23:58:05 INFO analysis.Analyzer: Max iterations (2) reached for batch CaseInsensitiveAttributeReferences
14/06/06 23:58:05 INFO analysis.Analyzer: Max iterations (2) reached for batch MultiInstanceRelations
14/06/06 23:58:05 INFO analysis.Analyzer: Max iterations (2) reached for batch CaseInsensitiveAttributeReferences
14/06/06 23:58:05 INFO metastore.HiveMetaStore: 0: get_table : db=default tbl=src
14/06/06 23:58:05 INFO HiveMetaStore.audit: ugi=rxin    ip=unknown-ip-addr  cmd=get_table : db=default tbl=src  
14/06/06 23:58:05 INFO storage.MemoryStore: ensureFreeSpace(147699) called with curMem=737503, maxMem=1145674137
14/06/06 23:58:05 INFO storage.MemoryStore: Block broadcast_5 stored as values to memory (estimated size 144.2 KB, free 1091.8 MB)
14/06/06 23:58:05 INFO sql.SQLContext$$anon$1: Max iterations (2) reached for batch Add exchange
14/06/06 23:58:05 INFO sql.SQLContext$$anon$1: Max iterations (2) reached for batch Prepare Expressions
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, tree:
ExplainCommandPhysical 
 Aggregate false, [key#12], [key#12,SUM(PartialCount#14L) AS c_1#10L]
  Exchange (HashPartitioning [key#12:0], 150)
   Aggregate true, [key#12], [key#12,COUNT(value#13) AS PartialCount#14L]
    HiveTableScan [key#12,value#13], (MetastoreRelation default, src, None), None

    at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:47)
    at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:265)
    at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildrenUp(TreeNode.scala:249)
    at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:215)
    at org.apache.spark.sql.execution.AddExchange$.apply(Exchange.scala:93)
    at org.apache.spark.sql.execution.AddExchange$.apply(Exchange.scala:89)
    at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$apply$1$$anonfun$apply$2.apply(RuleExecutor.scala:62)
    at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$apply$1$$anonfun$apply$2.apply(RuleExecutor.scala:60)
    at scala.collection.IndexedSeqOptimized$class.foldl(IndexedSeqOptimized.scala:51)
    at scala.collection.IndexedSeqOptimized$class.foldLeft(IndexedSeqOptimized.scala:60)
    at scala.collection.mutable.WrappedArray.foldLeft(WrappedArray.scala:34)
    at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$apply$1.apply(RuleExecutor.scala:60)
    at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$apply$1.apply(RuleExecutor.scala:52)
    at scala.collection.immutable.List.foreach(List.scala:318)
    at org.apache.spark.sql.catalyst.rules.RuleExecutor.apply(RuleExecutor.scala:52)
    at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:275)
    at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:275)
    at org.apache.spark.sql.hive.HiveContext$QueryExecution.toRdd$lzycompute(HiveContext.scala:260)
    at org.apache.spark.sql.hive.HiveContext$QueryExecution.toRdd(HiveContext.scala:248)
    at org.apache.spark.sql.hive.HiveContext.hiveql(HiveContext.scala:85)
    at org.apache.spark.sql.hive.HiveContext.hql(HiveContext.scala:90)
    at $i$$$$9579e5b89ab1eb428704b684e2e341c$$$$$.<init>(<console>:70)
    at $i$$$$9579e5b89ab1eb428704b684e2e341c$$$$$.<clinit>(<console>)
    at .<init>(<console>:7)
    at .<clinit>(<console>)
    at $print(<console>)
    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.lang.reflect.Method.invoke(Method.java:606)
    at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:734)
    at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:983)
    at scala.tools.nsc.interpreter.IMain.loadAndRunReq$1(IMain.scala:573)
    at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:604)
    at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:568)
    at scala.tools.nsc.interpreter.ILoop.reallyInterpret$1(ILoop.scala:760)
    at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:805)
    at scala.tools.nsc.interpreter.ILoop.command(ILoop.scala:717)
    at scala.tools.nsc.interpreter.ILoop.processLine$1(ILoop.scala:581)
    at scala.tools.nsc.interpreter.ILoop.innerLoop$1(ILoop.scala:588)
    at scala.tools.nsc.interpreter.ILoop.loop(ILoop.scala:591)
    at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply$mcZ$sp(ILoop.scala:882)
    at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:837)
    at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:837)
    at scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
    at scala.tools.nsc.interpreter.ILoop.process(ILoop.scala:837)
    at scala.tools.nsc.interpreter.ILoop.main(ILoop.scala:904)
    at xsbt.ConsoleInterface.run(ConsoleInterface.scala:69)
    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.lang.reflect.Method.invoke(Method.java:606)
    at sbt.compiler.AnalyzingCompiler.call(AnalyzingCompiler.scala:102)
    at sbt.compiler.AnalyzingCompiler.console(AnalyzingCompiler.scala:77)
    at sbt.Console.sbt$Console$$console0$1(Console.scala:23)
    at sbt.Console$$anonfun$apply$2$$anonfun$apply$1.apply$mcV$sp(Console.scala:24)
    at sbt.Console$$anonfun$apply$2$$anonfun$apply$1.apply(Console.scala:24)
    at sbt.Console$$anonfun$apply$2$$anonfun$apply$1.apply(Console.scala:24)
    at sbt.Logger$$anon$4.apply(Logger.scala:90)
    at sbt.TrapExit$App.run(TrapExit.scala:244)
    at java.lang.Thread.run(Thread.java:744)
Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Failed to copy node.  Is otherCopyArgs specified correctly for ExplainCommandPhysical?, tree:
ExplainCommandPhysical 
 Aggregate false, [key#12], [key#12,SUM(PartialCount#14L) AS c_1#10L]
  Exchange (HashPartitioning [key#12:0], 150)
   Aggregate true, [key#12], [key#12,COUNT(value#13) AS PartialCount#14L]
    HiveTableScan [key#12,value#13], (MetastoreRelation default, src, None), None

    at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:275)
    at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:266)
    at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:46)
    ... 60 more

@AmplabJenkins
Copy link

Merged build finished. All automated tests passed.

@AmplabJenkins
Copy link

All automated tests passed.
Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15523/

@marmbrus
Copy link
Contributor

marmbrus commented Jun 7, 2014

Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Failed to copy node. Is otherCopyArgs specified correctly for ExplainCommandPhysical?, tree:

Second parameter lists aren't automatically copied. You'll need to add override def otherCopyArgs = sc :: Nil

@concretevitamin
Copy link
Contributor Author

Fixed. Additionally, should output be a single column of something like "plan" w/ StringType?

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@marmbrus
Copy link
Contributor

marmbrus commented Jun 7, 2014

Yes, that sounds reasonable.

@rxin
Copy link
Contributor

rxin commented Jun 7, 2014

Can you add a test case?

@AmplabJenkins
Copy link

Merged build finished. All automated tests passed.

@AmplabJenkins
Copy link

All automated tests passed.
Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15528/

@concretevitamin
Copy link
Contributor Author

I added a test which is passing in this branch but failing in master.

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished. All automated tests passed.

@AmplabJenkins
Copy link

All automated tests passed.
Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15573/

@@ -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.

@asfgit asfgit closed this in a9ec033 Jun 9, 2014
asfgit pushed a commit that referenced this pull request Jun 9, 2014
This PR attempts to resolve [SPARK-1704](https://issues.apache.org/jira/browse/SPARK-1704) by introducing a physical plan for EXPLAIN commands, which just prints out the debug string (containing various SparkSQL's plans) of the corresponding QueryExecution for the actual query.

Author: Zongheng Yang <zongheng.y@gmail.com>

Closes #1003 from concretevitamin/explain-cmd and squashes the following commits:

5b7911f [Zongheng Yang] Add a regression test.
1bfa379 [Zongheng Yang] Modify output().
719ada9 [Zongheng Yang] Override otherCopyArgs for ExplainCommandPhysical.
4318fd7 [Zongheng Yang] Make all output one Row.
439c6ab [Zongheng Yang] Minor cleanups.
408f574 [Zongheng Yang] SPARK-1704: Add CommandStrategy and ExplainCommandPhysical.

(cherry picked from commit a9ec033)
Signed-off-by: Michael Armbrust <michael@databricks.com>
@marmbrus
Copy link
Contributor

marmbrus commented Jun 9, 2014

Merged into 1.0 and master. Thanks!

@concretevitamin concretevitamin deleted the explain-cmd branch June 9, 2014 23:51
pdeyhim pushed a commit to pdeyhim/spark-1 that referenced this pull request Jun 25, 2014
This PR attempts to resolve [SPARK-1704](https://issues.apache.org/jira/browse/SPARK-1704) by introducing a physical plan for EXPLAIN commands, which just prints out the debug string (containing various SparkSQL's plans) of the corresponding QueryExecution for the actual query.

Author: Zongheng Yang <zongheng.y@gmail.com>

Closes apache#1003 from concretevitamin/explain-cmd and squashes the following commits:

5b7911f [Zongheng Yang] Add a regression test.
1bfa379 [Zongheng Yang] Modify output().
719ada9 [Zongheng Yang] Override otherCopyArgs for ExplainCommandPhysical.
4318fd7 [Zongheng Yang] Make all output one Row.
439c6ab [Zongheng Yang] Minor cleanups.
408f574 [Zongheng Yang] SPARK-1704: Add CommandStrategy and ExplainCommandPhysical.
xiliu82 pushed a commit to xiliu82/spark that referenced this pull request Sep 4, 2014
This PR attempts to resolve [SPARK-1704](https://issues.apache.org/jira/browse/SPARK-1704) by introducing a physical plan for EXPLAIN commands, which just prints out the debug string (containing various SparkSQL's plans) of the corresponding QueryExecution for the actual query.

Author: Zongheng Yang <zongheng.y@gmail.com>

Closes apache#1003 from concretevitamin/explain-cmd and squashes the following commits:

5b7911f [Zongheng Yang] Add a regression test.
1bfa379 [Zongheng Yang] Modify output().
719ada9 [Zongheng Yang] Override otherCopyArgs for ExplainCommandPhysical.
4318fd7 [Zongheng Yang] Make all output one Row.
439c6ab [Zongheng Yang] Minor cleanups.
408f574 [Zongheng Yang] SPARK-1704: Add CommandStrategy and ExplainCommandPhysical.
@qiaohaijun
Copy link

+1

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants