You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/05/21 03:54:01 UTC

[GitHub] [spark] viirya opened a new pull request #24654: [SPARK-27439][SQL] Explainging Dataset should show correct resolved plans

viirya opened a new pull request #24654: [SPARK-27439][SQL] Explainging Dataset should show correct resolved plans
URL: https://github.com/apache/spark/pull/24654
 
 
   ## What changes were proposed in this pull request?
   
   Because a review is resolved during analysis when we create a dataset, the content of the view is determined when the dataset is created, not when it is evaluated. Now the explain result of a dataset is not correctly consistent with the collected result of it, because we use pre-analyzed logical plan of the dataset in explain command. The explain command will analyzed the logical plan passed in. So if a view is changed after the dataset was created, the plans shown by explain command aren't the same with the plan of the dataset.
   
   ```scala
   scala> spark.range(10).createOrReplaceTempView("test")
   scala> spark.range(5).createOrReplaceTempView("test2")
   scala> spark.sql("select * from test").createOrReplaceTempView("tmp001")
   scala> val df = spark.sql("select * from tmp001")
   scala> spark.sql("select * from test2").createOrReplaceTempView("tmp001")
   scala> df.show
   +---+
   | id|
   +---+
   |  0|
   |  1|
   |  2|
   |  3|
   |  4|
   |  5|
   |  6|
   |  7|
   |  8|
   |  9|
   +---+
   scala> df.explain(true)
   ```
   
   Before:
   ```scala
   == Parsed Logical Plan ==
   'Project [*]
   +- 'UnresolvedRelation `tmp001`
   
   == Analyzed Logical Plan ==
   id: bigint
   Project [id#2L]
   +- SubqueryAlias `tmp001`
      +- Project [id#2L]
         +- SubqueryAlias `test2`
            +- Range (0, 5, step=1, splits=Some(12))
   
   == Optimized Logical Plan ==
   Range (0, 5, step=1, splits=Some(12))
   
   == Physical Plan ==
   *(1) Range (0, 5, step=1, splits=12)
   ```
   
   After:
   ```scala
   == Parsed Logical Plan ==
   'Project [*]
   +- 'UnresolvedRelation `tmp001`
   
   == Analyzed Logical Plan ==
   id: bigint
   Project [id#0L]
   +- SubqueryAlias `tmp001`
      +- Project [id#0L]
         +- SubqueryAlias `test`
            +- Range (0, 10, step=1, splits=Some(12))
   
   == Optimized Logical Plan ==
   Range (0, 10, step=1, splits=Some(12))
   
   == Physical Plan ==
   *(1) Range (0, 10, step=1, splits=12)
   ```
   
   Previous PR to this issue has a regression when to explain an explain statement, like `sql("explain select 1").explain(true)`. This new fix is following up with @hvanhovell's advice at https://github.com/apache/spark/pull/24464#issuecomment-494165538.
   
   Explain an explain:
   ```scala
   scala> sql("explain select 1").explain(true)
   == Parsed Logical Plan ==
   ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false
   
   == Analyzed Logical Plan ==
   plan: string
   ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false
   
   == Optimized Logical Plan ==
   ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false
   
   == Physical Plan ==
   Execute ExplainCommand
      +- ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false                                                               
   ```
   
   Btw, I found there is a regression after applying @hvanhovell's advice:
   
   ```scala
   spark.readStream
         .format("org.apache.spark.sql.streaming.test")
         .load()
         .explain(true)
   ```
   
   ```scala
   == Parsed Logical Plan ==                                                                                                                              
   StreamingRelation DataSource(org.apache.spark.sql.test.TestSparkSession@3e8c7175,org.apache.spark.sql.streaming.test,List(),None,List(),None,Map(),None
   ), dummySource, [a#559]                                                                                                                                
                                                                                                                                                          
   == Analyzed Logical Plan ==                                                                                                                            
   a: int
   StreamingRelation DataSource(org.apache.spark.sql.test.TestSparkSession@3e8c7175,org.apache.spark.sql.streaming.test,List(),None,List(),None,Map(),Non$
   ), dummySource, [a#559]
   
   == Optimized Logical Plan ==
   org.apache.spark.sql.AnalysisException: Queries with streaming sources must be executed with writeStream.start();;
   dummySource
   == Physical Plan ==
   org.apache.spark.sql.AnalysisException: Queries with streaming sources must be executed with writeStream.start();;
   dummySource
   ```
   
   So I did a change to that to fix it too.
   
   ## How was this patch tested?
   
   Added test and manually test.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org