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 2021/04/28 14:49:20 UTC

[GitHub] [spark] tgravescs commented on a change in pull request #31756: [SPARK-34637] [SQL] Support DPP + AQE when the broadcast exchange can be reused

tgravescs commented on a change in pull request #31756:
URL: https://github.com/apache/spark/pull/31756#discussion_r622256598



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveDynamicPruningFilters.scala
##########
@@ -41,15 +42,26 @@ case class PlanAdaptiveDynamicPruningFilters(
           adaptivePlan: AdaptiveSparkPlanExec), exprId, _)) =>
         val packedKeys = BindReferences.bindReferences(
           HashJoin.rewriteKeyExpr(buildKeys), adaptivePlan.executedPlan.output)
-        val mode = HashedRelationBroadcastMode(packedKeys)
-        // plan a broadcast exchange of the build side of the join
-        val exchange = BroadcastExchangeExec(mode, adaptivePlan.executedPlan)
-        val existingStage = stageCache.get(exchange.canonicalized)
-        if (existingStage.nonEmpty && conf.exchangeReuseEnabled) {
-          val name = s"dynamicpruning#${exprId.id}"
-          val reuseQueryStage = existingStage.get.newReuseInstance(0, exchange.output)
-          val broadcastValues =
-            SubqueryBroadcastExec(name, index, buildKeys, reuseQueryStage)
+
+        val canReuseExchange = conf.exchangeReuseEnabled && buildKeys.nonEmpty &&
+          plan.find {
+            case BroadcastHashJoinExec(_, _, _, BuildLeft, _, left, _, _) =>
+              left.sameResult(adaptivePlan.executedPlan)
+            case BroadcastHashJoinExec(_, _, _, BuildRight, _, _, right, _) =>
+              right.sameResult(adaptivePlan.executedPlan)
+            case _ => false
+          }.isDefined
+
+        if(canReuseExchange) {
+          val mode = HashedRelationBroadcastMode(packedKeys)
+          // plan a broadcast exchange of the build side of the join
+          val exchange = BroadcastExchangeExec(mode, adaptivePlan.executedPlan)
+          exchange.setLogicalLink(adaptivePlan.executedPlan.logicalLink.get)
+          val newAdaptivePlan = AdaptiveSparkPlanExec(

Review comment:
       I'm trying to understand this change.  Could you add more description to the PR as to what you are doing and how this solves the problem?
   
   It seems now we create a AdaptiveSparkPlanExec inside another AdaptiveSparkPlanExec where that one just does the broadcast. I guess that one has the DPP already so then the existing logic checking stagecache will reuse the DPP one after that?




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



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