JkSelf commented on a change in pull request #26434: [SPARK-29544] [SQL] 
optimize skewed partition based on data size
URL: https://github.com/apache/spark/pull/26434#discussion_r354111510
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala
 ##########
 @@ -0,0 +1,281 @@
+/*
+ * 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 scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration.Duration
+
+import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, SparkEnv}
+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._
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, 
UnknownPartitioning}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.joins.SortMergeJoinExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.ThreadUtils
+
+case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private val supportedJoinTypes =
+    Inner :: Cross :: LeftSemi :: LeftAnti :: LeftOuter :: RightOuter :: Nil
+
+  /**
+   * A partition is considered as a skewed partition if its size is larger 
than the median
+   * partition size * spark.sql.adaptive.skewedPartitionFactor and also larger 
than
+   * spark.sql.adaptive.skewedPartitionSizeThreshold.
+   */
+  private def isSkewed(
+     stats: MapOutputStatistics,
+     partitionId: Int,
+     medianSize: Long): Boolean = {
+    val size = stats.bytesByPartitionId(partitionId)
+    size > medianSize * conf.adaptiveSkewedFactor &&
+      size > conf.adaptiveSkewedSizeThreshold
+  }
+
+  private def medianSize(stats: MapOutputStatistics): Long = {
+    val bytesLen = stats.bytesByPartitionId.length
+    val bytes = stats.bytesByPartitionId.sorted
+    if (bytes(bytesLen / 2) > 0) bytes(bytesLen / 2) else 1
+  }
+
+  /*
+  * Get all the map data size for specific reduce partitionId.
+  */
+  def getMapSizeForSpecificPartition(partitionId: Int, shuffleId: Int): 
Array[Long] = {
+    val mapOutputTracker = 
SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]
+    mapOutputTracker.shuffleStatuses.get(shuffleId).
+      get.mapStatuses.map{_.getSizeForBlock(partitionId)}
+  }
+
+  /*
+  * Split the mappers based on the map size of specific skewed reduce 
partitionId.
+  */
+  def splitMappersBasedDataSize(mapPartitionSize: Array[Long], numMappers: 
Int): Array[Int] = {
+    val advisoryTargetPostShuffleInputSize = conf.targetPostShuffleInputSize
+    val partitionStartIndices = ArrayBuffer[Int]()
+    var i = 0
+    var postMapPartitionSize: Long = mapPartitionSize(i)
+    partitionStartIndices += i
+    while (i < numMappers && i + 1 < numMappers) {
+      val nextIndex = if (i + 1 < numMappers) {
+        i + 1
+      } else numMappers -1
+
+      if (postMapPartitionSize + mapPartitionSize(nextIndex) > 
advisoryTargetPostShuffleInputSize) {
+        postMapPartitionSize = mapPartitionSize(nextIndex)
+        partitionStartIndices += nextIndex
+      } else {
+        postMapPartitionSize += mapPartitionSize(nextIndex)
+      }
+      i += 1
+    }
+    partitionStartIndices.toArray
+  }
+
+  /**
+   * We split the partition into several splits. Each split reads the data 
from several map outputs
+   * ranging from startMapId to endMapId(exclusive). This method calculates 
the split number and
+   * the startMapId for all splits.
+   */
+  private def estimateMapIdStartIndices(
+    stage: QueryStageExec,
+    partitionId: Int,
+    medianSize: Long): Array[Int] = {
+    val dependency = getShuffleStage(stage).plan.shuffleDependency
+    val shuffleId = dependency.shuffleHandle.shuffleId
+    val mapSize = getMapSizeForSpecificPartition(partitionId, shuffleId)
+    val numMappers = dependency.rdd.partitions.length
+    splitMappersBasedDataSize(mapSize, numMappers)
+  }
+
+  private def getShuffleStage(queryStage: QueryStageExec): 
ShuffleQueryStageExec = {
+    queryStage match {
+      case stage: ShuffleQueryStageExec => stage
+      case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage
+    }
+  }
+
+  private def getStatistics(queryStage: QueryStageExec): MapOutputStatistics = 
{
+    val shuffleStage = queryStage match {
+      case stage: ShuffleQueryStageExec => stage
+      case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage
+    }
+    val metrics = shuffleStage.mapOutputStatisticsFuture
+    assert(metrics.isCompleted, "ShuffleQueryStageExec should already be 
ready")
+    ThreadUtils.awaitResult(metrics, Duration.Zero)
+  }
+
+  /**
+   * Base optimization support check: the join type is supported and plan 
statistics is available.
+   * Note that for some join types(like left outer), whether a certain 
partition can be optimized
+   * also depends on the filed isSkewAndSupportsSplit.
+   */
+  private def supportOptimization(
+    joinType: JoinType,
+    leftStage: QueryStageExec,
+    rightStage: QueryStageExec): Boolean = {
+    val joinTypeSupported = supportedJoinTypes.contains(joinType)
+    val shuffleStageCheck = 
ShuffleQueryStageExec.isShuffleQueryStageExec(leftStage) &&
+      ShuffleQueryStageExec.isShuffleQueryStageExec(rightStage)
+    val statisticsReady: Boolean = if (shuffleStageCheck) {
+      getStatistics(leftStage) != null && getStatistics(rightStage) != null
 
 Review comment:
   it seems not be null. Wrong understanding about the leftStage and rightStage 
may not done simultaneously.

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