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

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala
 ##########
 @@ -0,0 +1,261 @@
+/*
+ * 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.concurrent.duration.Duration
+
+import org.apache.spark.MapOutputStatistics
+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
+  }
+
+  /**
+   * To equally divide n elements into m buckets, basically each bucket should 
have n/m elements,
+   * for the remaining n%m elements, add one more element to the first n%m 
buckets each. Returns
+   * a sequence with length numBuckets and each value represents the start 
index of each bucket.
+   */
+  def equallyDivide(numElements: Int, numBuckets: Int): Seq[Int] = {
+    val elementsPerBucket = numElements / numBuckets
+    val remaining = numElements % numBuckets
+    val splitPoint = (elementsPerBucket + 1) * remaining
+    (0 until remaining).map(_ * (elementsPerBucket + 1)) ++
+      (remaining until numBuckets).map(i => splitPoint + (i - remaining) * 
elementsPerBucket)
+  }
+
+  /**
+   * 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 metrics = getStatistics(stage)
+    val size = metrics.bytesByPartitionId(partitionId)
+    val factor = size / medianSize
+    val numMappers = getShuffleStage(stage).
+      plan.shuffleDependency.rdd.partitions.length
 
 Review comment:
   we can easily get the data size of each mapper, shall we split mapper ranges 
based on data size?

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to