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-27439][SQL] Explainging Dataset should show correct resolved plans #24464

Closed
wants to merge 4 commits into from

Conversation

viirya
Copy link
Member

@viirya viirya commented Apr 26, 2019

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> 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:

== 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:

== 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)

To fix it, this passes query execution of Dataset when explaining it. The query execution contains pre-analyzed plan which is consistent with Dataset's result.

How was this patch tested?

Manually test and unit test.

@viirya
Copy link
Member Author

viirya commented Apr 26, 2019

@SparkQA
Copy link

SparkQA commented Apr 26, 2019

Test build #104923 has finished for PR 24464 at commit dcbcc7f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

*/
case class ExplainCommand(
logicalPlan: LogicalPlan,
extended: Boolean = false,
codegen: Boolean = false,
cost: Boolean = false)
cost: Boolean = false,
optQueryExecution: Option[QueryExecution] = None)
Copy link
Member Author

Choose a reason for hiding this comment

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

I don't have a better way to keep using analyzed plan of dataset and showing correct pre-analyzed plan, other than passing in the query execution of the dataset.

@viirya
Copy link
Member Author

viirya commented Apr 29, 2019

@cloud-fan @dongjoon-hyun @HyukjinKwon Do you think this fix work? Please take a look. Thanks.

@@ -498,7 +498,8 @@ class Dataset[T] private[sql](
* @since 1.6.0
*/
def explain(extended: Boolean): Unit = {
val explain = ExplainCommand(queryExecution.logical, extended = extended)
val explain = ExplainCommand(queryExecution.logical, extended = extended,
Copy link
Contributor

Choose a reason for hiding this comment

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

what about passing the QueryExecution as first parameter? IIUC, there is only another place where ExplainCommand is created, so it is not going to be a big change too and it is going to be cleaner IMO...

Copy link
Member Author

Choose a reason for hiding this comment

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

ExplainCommand is also created in SparkSqlParser where there is no QueryExecution. So it has both LogicalPlan and QueryExecution parameters.

Copy link
Contributor

@mgaido91 mgaido91 May 1, 2019

Choose a reason for hiding this comment

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

Yes, my point is: can we create a QueryExecution in SparkSqlParser and pass that to ExplainCommand?

The only problem I can see in doing this is that we need to bind the newly generated QueryExecution to a SparkSession.

Copy link
Member Author

Choose a reason for hiding this comment

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

Made a change. Please see if it is clearer for you.

@@ -494,11 +494,15 @@ class Dataset[T] private[sql](
/**
* Prints the plans (logical and physical) to the console for debugging purposes.
*
* Note that temporary views are already resolved when creating `Dataset`. So if
* temporary views are changed after that, the output of this command shows the plans
* before such changes.
Copy link
Member Author

Choose a reason for hiding this comment

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

I think it is clearer to make a note in this doc.

Copy link
Contributor

Choose a reason for hiding this comment

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

I am not sure whether this is the right place for this comment. I mean, I think that putting it here, it is not clear whether the output of the explain shows the plan with "old" views, but the dataset is executed with the "new" ones or both use the "old", as it is.

Copy link
Member Author

Choose a reason for hiding this comment

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

This is the main access of explain command for Dataset. If not this place, any other place you will suggest?

Copy link
Contributor

Choose a reason for hiding this comment

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

My point is: this note and this behavior is not specific to explain, but it is common to all operations on a dataset. Putting the comment here can be confusing as it may imply that it is not true for other operations on dataset, which is not true. I'd rather put it in the comment of the Dataset class.

* @param extended whether to do extended explain or not
* @param codegen whether to output generated code from whole-stage codegen or not
* @param cost whether to show cost information for operators.
*/
case class ExplainCommand(
logicalPlan: LogicalPlan,
queryExecution: QueryExecution,
Copy link
Member Author

Choose a reason for hiding this comment

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

@mgaido91 ExplainCommand now accepts only QueryExecution.

/**
* This is mainly used for tests.
*/
def apply(
Copy link
Member Author

Choose a reason for hiding this comment

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

This is for test only. Because some test suites passes logicalPlan and extended, and we can't overload apply with default parameters. So to create this for test only purpose.

@@ -494,11 +494,15 @@ class Dataset[T] private[sql](
/**
* Prints the plans (logical and physical) to the console for debugging purposes.
*
* Note that temporary views are already resolved when creating `Dataset`. So if
* temporary views are changed after that, the output of this command shows the plans
* before such changes.
Copy link
Contributor

Choose a reason for hiding this comment

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

I am not sure whether this is the right place for this comment. I mean, I think that putting it here, it is not clear whether the output of the explain shows the plan with "old" views, but the dataset is executed with the "new" ones or both use the "old", as it is.

extended: Boolean,
codegen: Boolean,
cost: Boolean): ExplainCommand = {
val sparkSession = SparkSession.getActiveSession
Copy link
Contributor

Choose a reason for hiding this comment

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

SparkSession.active?

@SparkQA
Copy link

SparkQA commented May 1, 2019

Test build #105054 has finished for PR 24464 at commit 2cb7d26.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya viirya changed the title [SPARK-27439][SQL] Use existing queryexecution in explaining Dataset [SPARK-27439][SQL] Explainging Dataset should show correct resolved plans May 2, 2019
@viirya
Copy link
Member Author

viirya commented May 2, 2019

@mgaido91 Your comments were addressed. Thanks for review.

@viirya
Copy link
Member Author

viirya commented May 2, 2019

@cloud-fan @dongjoon-hyun Can you help check this? I think it should be in better shape now.

@SparkQA
Copy link

SparkQA commented May 2, 2019

Test build #105076 has finished for PR 24464 at commit 18064d0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented May 2, 2019

Test build #105077 has finished for PR 24464 at commit bfd6eea.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@@ -498,7 +498,7 @@ class Dataset[T] private[sql](
* @since 1.6.0
*/
def explain(extended: Boolean): Unit = {
val explain = ExplainCommand(queryExecution.logical, extended = extended)
val explain = ExplainCommand(queryExecution, extended = extended)
Copy link
Member

Choose a reason for hiding this comment

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

To fix the current bug, I agree with @viirya . I believe this is inevitable.
Could you give us some guide for this, @cloud-fan and @gatorsmile ?

@dongjoon-hyun
Copy link
Member

Retest this please.

@SparkQA
Copy link

SparkQA commented May 3, 2019

Test build #105105 has finished for PR 24464 at commit bfd6eea.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@mgaido91 mgaido91 left a comment

Choose a reason for hiding this comment

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

LGTM apart a couple of style-related comments, thanks for this fix @viirya

* run through the analyzer and optimizer when this command is actually run.
*/
def apply(
logicalPlan: LogicalPlan,
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: one more indent here and in the lines below?

* This is mainly used for tests.
*/
def apply(
logicalPlan: LogicalPlan,
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: ditto

@SparkQA
Copy link

SparkQA commented May 6, 2019

Test build #105137 has finished for PR 24464 at commit 62959cd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

Thank you, @viirya and @mgaido91 .
In order to fix the original problem, I'll merge this second trial patch.

@gatorsmile
Copy link
Member

If the analyze doesn't finish within maxIterations, the RuleExecutor try to print the whole plan, and it finally will try to print QueryExecution.optimizedPlan and QueryExecution.executedPlan, which would trigger assertNotAnalysisRule and throws RuntimeException This method should not be called in the analyzer.

Could we revert this commit?

@hvanhovell
Copy link
Contributor

hvanhovell commented May 20, 2019

@viirya can we just fix this by doing the following:

def explain(extended: Boolean): Unit = {
  val outputString =
     if (extended) {
       queryExecution.toString
     } else {
       queryExecution.simpleString
     }
  // scalastyle:off println
  println(outputString)
  // scalastyle:on println
}

That seems a lot simpler, and it does not trigger full analysis and optimization from the parser.

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented May 20, 2019

Thank you, @gatorsmile and @hvanhovell .
@gatorsmile also gave the regression example about explain on EXPLAIN statement. Even a simple example like the following, the output is a huge mess.

scala> sql("explain select 1").explain(true)
== Parsed Logical Plan ==
ExplainCommand == Parsed Logical Plan ==
'Project [unresolvedalias(1, None)]
+- OneRowRelation

== Analyzed Logical Plan ==
1: int
Project [1 AS 1#2]
+- OneRowRelation

== Optimized Logical Plan ==
Project [1 AS 1#2]
+- OneRowRelation

== Physical Plan ==
*(1) Project [1 AS 1#2]
+- *(1) Scan OneRowRelation[]
, false, false, false

== Analyzed Logical Plan ==
plan: string
ExplainCommand == Parsed Logical Plan ==
'Project [unresolvedalias(1, None)]
+- OneRowRelation

== Analyzed Logical Plan ==
1: int
Project [1 AS 1#2]
+- OneRowRelation

== Optimized Logical Plan ==
Project [1 AS 1#2]
+- OneRowRelation

== Physical Plan ==
*(1) Project [1 AS 1#2]
+- *(1) Scan OneRowRelation[]
, false, false, false

== Optimized Logical Plan ==
ExplainCommand == Parsed Logical Plan ==
'Project [unresolvedalias(1, None)]
+- OneRowRelation

== Analyzed Logical Plan ==
1: int
Project [1 AS 1#2]
+- OneRowRelation

== Optimized Logical Plan ==
Project [1 AS 1#2]
+- OneRowRelation

== Physical Plan ==
*(1) Project [1 AS 1#2]
+- *(1) Scan OneRowRelation[]
, false, false, false

== Physical Plan ==
Execute ExplainCommand
   +- ExplainCommand == Parsed Logical Plan ==
'Project [unresolvedalias(1, None)]
+- OneRowRelation

== Analyzed Logical Plan ==
1: int
Project [1 AS 1#2]
+- OneRowRelation

== Optimized Logical Plan ==
Project [1 AS 1#2]
+- OneRowRelation

== Physical Plan ==
*(1) Project [1 AS 1#2]
+- *(1) Scan OneRowRelation[]
, false, false, false

Sorry about this mess. I'll revert this. @viirya . Please proceed with @hvanhovell 's advice~

@dongjoon-hyun
Copy link
Member

This is reverted via 039db87 and I reopened SPARK-27439 .

@viirya
Copy link
Member Author

viirya commented May 21, 2019

oh sorry for that and thanks @gatorsmile, @hvanhovell and @dongjoon-hyun
I will follow up with @hvanhovell's advice.

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