This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new cd2ef9c  [SPARK-35567][SQL] Fix: Explain cost is not showing 
statistics for all the nodes
cd2ef9c is described below

commit cd2ef9cb43f4e27302906ac2f605df4b66a72a2f
Author: shahid <shahidk...@gmail.com>
AuthorDate: Tue Jun 1 00:55:29 2021 +0800

    [SPARK-35567][SQL] Fix: Explain cost is not showing statistics for all the 
nodes
    
    ### What changes were proposed in this pull request?
    Explain cost command in spark currently doesn't show statistics for all the 
nodes. It misses some nodes in almost all the TPCDS queries.
    In this PR, we are collecting all the plan nodes including the subqueries 
and computing  the statistics for each node, if it doesn't exists in stats 
cache,
    
    ### Why are the changes needed?
    **Before Fix**
    For eg: Query1,  Project node doesn't have statistics
    
![image](https://user-images.githubusercontent.com/23054875/120123442-868feb00-c1cc-11eb-9af9-3a87bf2117d2.png)
    
    Query15, Aggregate node doesn't have statistics
    
    
![image](https://user-images.githubusercontent.com/23054875/120123296-a4108500-c1cb-11eb-89df-7fddd651572e.png)
    
    **After Fix:**
    Query1:
    
![image](https://user-images.githubusercontent.com/23054875/120123559-1df53e00-c1cd-11eb-938a-53704f5240e6.png)
    Query 15:
    
![image](https://user-images.githubusercontent.com/23054875/120123665-bb507200-c1cd-11eb-8ea2-84c732215bac.png)
    ### Does this PR introduce _any_ user-facing change?
    No
    
    ### How was this patch tested?
    Manual testing
    
    Closes #32704 from shahidki31/shahid/fixshowstats.
    
    Authored-by: shahid <shahidk...@gmail.com>
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
---
 .../org/apache/spark/sql/execution/QueryExecution.scala    | 14 ++++++--------
 1 file changed, 6 insertions(+), 8 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index 0795776..247baea 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -28,7 +28,6 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
 import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker}
 import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
-import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
 import org.apache.spark.sql.catalyst.expressions.codegen.ByteCodeStats
 import org.apache.spark.sql.catalyst.plans.QueryPlan
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
@@ -256,13 +255,12 @@ class QueryExecution(
 
     // trigger to compute stats for logical plans
     try {
-      optimizedPlan.foreach(_.expressions.foreach(_.foreach {
-        case subqueryExpression: SubqueryExpression =>
-          // trigger subquery's child plan stats propagation
-          subqueryExpression.plan.stats
-        case _ =>
-      }))
-      optimizedPlan.stats
+      // This will trigger to compute stats for all the nodes in the plan, 
including subqueries,
+      // if the stats doesn't exist in the statsCache and update the 
statsCache corresponding
+      // to the node.
+      optimizedPlan.collectWithSubqueries {
+        case plan => plan.stats
+      }
     } catch {
       case e: AnalysisException => append(e.toString + "\n")
     }

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to