cloud-fan 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_r332436377
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala
 ##########
 @@ -0,0 +1,106 @@
+/*
+ * 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.{LeafExecNode, SparkPlan}
+import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, 
BuildRight}
+import org.apache.spark.sql.internal.SQLConf
+
+case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] {
+
+  def canUseLocalShuffleReaderLeft(join: BroadcastHashJoinExec): Boolean = {
+    join.buildSide == BuildLeft &&  
ShuffleQueryStageExec.isShuffleQueryStageExec(join.right)
+  }
+
+  def canUseLocalShuffleReaderRight(join: BroadcastHashJoinExec): Boolean = {
+    join.buildSide == BuildRight &&  
ShuffleQueryStageExec.isShuffleQueryStageExec(join.left)
+  }
+
+  override def apply(plan: SparkPlan): SparkPlan = {
+    if (!conf.optimizedLocalShuffleReaderEnabled) {
+      return plan
+    }
+
+    plan.transformDown{
+      case join: BroadcastHashJoinExec if canUseLocalShuffleReaderLeft(join) =>
+        val localReader = 
LocalShuffleReaderExec(join.right.asInstanceOf[QueryStageExec])
+        join.copy(right = localReader)
+      case join: BroadcastHashJoinExec if canUseLocalShuffleReaderRight(join) 
=>
+        val localReader = 
LocalShuffleReaderExec(join.left.asInstanceOf[QueryStageExec])
+        join.copy(left = localReader)
+    }
+  }
 
 Review comment:
   I spend more time to think about the cost framework, and I think we can't 
fully rely on it right now.
   
   For example, if there is a `rule1` doing something good, and a `rule2` doing 
something negative, the current cost framework in AQE simply discard the entire 
re-optimization and neither `rule1` or `rule2` takes effect. What we expect to 
see is to have `rule1` take effect while `rule2` does not.
   
   I think each rule should try its best to make sure itself is always 
positive. For this particular rule, we can run `EnsureRequirements` on the 
original plan and the updated plan, and only keep the updated plan if number of 
shuffles doesn't increase.

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