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_r358056077
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala
 ##########
 @@ -0,0 +1,313 @@
+/*
+ * 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.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR) &&
+      size > 
conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD)
+  }
+
+  private def medianSize(stats: MapOutputStatistics): Long = {
+    val numPartitions = stats.bytesByPartitionId.length
+    val bytes = stats.bytesByPartitionId.sorted
+    if (bytes(numPartitions / 2) > 0) bytes(numPartitions / 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 partition into the number of mappers. Each split read data from 
each mapper.
+   */
+  private def estimateMapStartIndices(
+      stage: QueryStageExec,
+      partitionId: Int,
+      medianSize: Long): Array[Int] = {
+    val dependency = 
ShuffleQueryStageExec.getShuffleStage(stage).plan.shuffleDependency
+    val numMappers = dependency.rdd.partitions.length
+    // TODO: split the partition based on the size
+    (0 until numMappers).toArray
+  }
+
+  private def getStatistics(queryStage: QueryStageExec): MapOutputStatistics = 
{
+    val shuffleStage = ShuffleQueryStageExec.getShuffleStage(queryStage)
+    val metrics = shuffleStage.plan.mapOutputStatisticsFuture
+    assert(metrics.isCompleted,
+      "ShuffleQueryStageExec should already be ready when executing 
OptimizeSkewedPartitions rule")
+    ThreadUtils.awaitResult(metrics, Duration.Zero)
+  }
+
+  /**
+   * Base optimization support check: the join type is supported.
+   * Note that for some join types(like left outer), whether a certain 
partition can be optimized
+   * also depends on the filed isSkewAndSupportsSplit.
 
 Review comment:
   `depends on which side is skewed`

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