JkSelf commented on a change in pull request #25295: [SPARK-28560][SQL] 
Optimize shuffle reader to local shuffle reader when smj converted to bhj in 
adaptive execution
URL: https://github.com/apache/spark/pull/25295#discussion_r326443605
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizedLocalShuffleReader.scala
 ##########
 @@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.adaptive
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, 
UnknownPartitioning}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.exchange.{EnsureRequirements, 
ShuffleExchangeExec}
+import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec}
+import org.apache.spark.sql.internal.SQLConf
+
+case class OptimizedLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private def setIsLocalToFalse(shuffleStage: QueryStageExec): QueryStageExec 
= {
+    shuffleStage match {
+      case stage: ShuffleQueryStageExec =>
+        stage.isLocalShuffle = false
+      case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) =>
+        stage.isLocalShuffle = false
+    }
+    shuffleStage
+  }
+
+  private def revertLocalShuffleReader(newPlan: SparkPlan): SparkPlan = {
+    val revertPlan = newPlan.transformUp {
+      case localReader: LocalShuffleReaderExec
+        if (ShuffleQueryStageExec.isShuffleQueryStageExec(localReader.child)) 
=>
+        setIsLocalToFalse(localReader.child)
+    }
+    revertPlan
+  }
+
+  override def apply(plan: SparkPlan): SparkPlan = {
+    // Collect the `BroadcastHashJoinExec` nodes and if isEmpty directly 
return.
+    val bhjs = plan.collect {
+      case bhj: BroadcastHashJoinExec => bhj
+    }
+
+    if (!conf.optimizedLocalShuffleReaderEnabled || bhjs.isEmpty) {
+      return plan
+    }
+
+    // If the streamedPlan is `ShuffleQueryStageExec`, set the value of 
`isLocalShuffle` to true
+    bhjs.map {
+      case bhj: BroadcastHashJoinExec =>
+        bhj.children map {
+          case stage: ShuffleQueryStageExec => stage.isLocalShuffle = true
+          case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) =>
+            stage.isLocalShuffle = true
+          case plan: SparkPlan => plan
+        }
+    }
+
+    // Add the new `LocalShuffleReaderExec` node if the value of 
`isLocalShuffle` is true
+    val newPlan = plan.transformUp {
+      case stage: ShuffleQueryStageExec if (stage.isLocalShuffle) =>
+        LocalShuffleReaderExec(stage)
+      case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) if 
(stage.isLocalShuffle) =>
+        LocalShuffleReaderExec(stage)
+    }
+
+    val afterEnsureRequirements = EnsureRequirements(conf).apply(newPlan)
+    val numExchanges = afterEnsureRequirements.collect {
+      case e: ShuffleExchangeExec => e
+    }.length
+    if (numExchanges > 0) {
+      logWarning("Local shuffle reader optimization is not applied due" +
+        " to additional shuffles will be introduced.")
+      revertLocalShuffleReader(newPlan)
+    } else {
+      newPlan
+    }
+  }
+}
+
+case class LocalShuffleReaderExec(
+    child: QueryStageExec) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = child.output
+
+  override def doCanonicalize(): SparkPlan = child.canonicalized
+
+  override def outputPartitioning: Partitioning = {
 
 Review comment:
   Here for local shuffle reader, the partition number of Partitioning is the 
number of mappers. and the partition number of child.outputPartitioning is the 
number of reducers.  So how can be `child.outputPartitioning` ?

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

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

Reply via email to