You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2022/06/04 06:13:09 UTC

[spark] branch branch-3.3 updated: [SPARK-39259][SQL][FOLLOWUP] Fix source and binary incompatibilities in transformDownWithSubqueries

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

maxgekk pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new b7e95bad882 [SPARK-39259][SQL][FOLLOWUP] Fix source and binary incompatibilities in transformDownWithSubqueries
b7e95bad882 is described below

commit b7e95bad882482168b7dd301fcfa3daf80477a7a
Author: Josh Rosen <jo...@databricks.com>
AuthorDate: Sat Jun 4 09:12:42 2022 +0300

    [SPARK-39259][SQL][FOLLOWUP] Fix source and binary incompatibilities in transformDownWithSubqueries
    
    ### What changes were proposed in this pull request?
    
    This is a followup to #36654. That PR modified the existing `QueryPlan.transformDownWithSubqueries` to add additional arguments for tree pattern pruning.
    
    In this PR, I roll back the change to that method's signature and instead add a new `transformDownWithSubqueriesAndPruning` method.
    
    ### Why are the changes needed?
    
    The original change breaks binary and source compatibility in Catalyst. Technically speaking, Catalyst APIs are considered internal to Spark and are subject to change between minor releases (see [source](https://github.com/apache/spark/blob/bb51add5c79558df863d37965603387d40cc4387/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala#L20-L24)), but I think it's nice to try to avoid API breakage when possible.
    
    While trying to compile some custom Catalyst code, I ran into issues when trying to call the `transformDownWithSubqueries` method without supplying a tree pattern filter condition. If I do `transformDownWithSubqueries() { f} ` then I get a compilation error. I think this is due to the first parameter group containing all default parameters.
    
    My PR's solution of adding a new `transformDownWithSubqueriesAndPruning` method solves this problem. It's also more consistent with the naming convention used for other pruning-enabled tree transformation methods.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    Existing tests.
    
    Closes #36765 from JoshRosen/SPARK-39259-binary-compatibility-followup.
    
    Authored-by: Josh Rosen <jo...@databricks.com>
    Signed-off-by: Max Gekk <ma...@gmail.com>
    (cherry picked from commit eda6c4b9987f0515cb0aae4686c8a0ae0a3987d4)
    Signed-off-by: Max Gekk <ma...@gmail.com>
---
 .../sql/catalyst/optimizer/finishAnalysis.scala    |  2 +-
 .../spark/sql/catalyst/plans/QueryPlan.scala       | 22 ++++++++++++++++------
 2 files changed, 17 insertions(+), 7 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
index 242c799dd22..a33069051d9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
@@ -84,7 +84,7 @@ object ComputeCurrentTime extends Rule[LogicalPlan] {
       treePatternbits.containsPattern(CURRENT_LIKE)
     }
 
-    plan.transformDownWithSubqueries(transformCondition) {
+    plan.transformDownWithSubqueriesAndPruning(transformCondition) {
       case subQuery =>
         subQuery.transformAllExpressionsWithPruning(transformCondition) {
           case cd: CurrentDate =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index d0283f4d367..cc62c81b101 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -454,7 +454,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]]
    * to rewrite the whole plan, include its subqueries, in one go.
    */
   def transformWithSubqueries(f: PartialFunction[PlanType, PlanType]): PlanType =
-    transformDownWithSubqueries(AlwaysProcess.fn, UnknownRuleId)(f)
+    transformDownWithSubqueries(f)
 
   /**
    * Returns a copy of this node where the given partial function has been recursively applied
@@ -479,10 +479,20 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]]
    * first to this node, then this node's subqueries and finally this node's children.
    * When the partial function does not apply to a given node, it is left unchanged.
    */
-  def transformDownWithSubqueries(
-    cond: TreePatternBits => Boolean = AlwaysProcess.fn, ruleId: RuleId = UnknownRuleId)
-    (f: PartialFunction[PlanType, PlanType])
-: PlanType = {
+  def transformDownWithSubqueries(f: PartialFunction[PlanType, PlanType]): PlanType = {
+    transformDownWithSubqueriesAndPruning(AlwaysProcess.fn, UnknownRuleId)(f)
+  }
+
+  /**
+   * This method is the top-down (pre-order) counterpart of transformUpWithSubqueries.
+   * Returns a copy of this node where the given partial function has been recursively applied
+   * first to this node, then this node's subqueries and finally this node's children.
+   * When the partial function does not apply to a given node, it is left unchanged.
+   */
+  def transformDownWithSubqueriesAndPruning(
+      cond: TreePatternBits => Boolean,
+      ruleId: RuleId = UnknownRuleId)
+      (f: PartialFunction[PlanType, PlanType]): PlanType = {
     val g: PartialFunction[PlanType, PlanType] = new PartialFunction[PlanType, PlanType] {
       override def isDefinedAt(x: PlanType): Boolean = true
 
@@ -490,7 +500,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]]
         val transformed = f.applyOrElse[PlanType, PlanType](plan, identity)
         transformed transformExpressionsDown {
           case planExpression: PlanExpression[PlanType] =>
-            val newPlan = planExpression.plan.transformDownWithSubqueries(cond, ruleId)(f)
+            val newPlan = planExpression.plan.transformDownWithSubqueriesAndPruning(cond, ruleId)(f)
             planExpression.withNewPlan(newPlan)
         }
       }


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