From ea18aa62ab4ef6ab0af55a9e330e224c7d0d853e Mon Sep 17 00:00:00 2001 From: jiake Date: Fri, 8 Nov 2019 14:34:20 +0800 Subject: [PATCH 01/23] optimize skewed partition based on data size --- .../org/apache/spark/MapOutputTracker.scala | 108 +++++++- .../apache/spark/shuffle/ShuffleManager.scala | 12 + .../shuffle/sort/SortShuffleManager.scala | 14 + .../apache/spark/sql/internal/SQLConf.scala | 37 +++ .../spark/sql/execution/ShuffledRowRDD.scala | 18 +- .../adaptive/AdaptiveSparkPlanExec.scala | 4 + .../adaptive/LocalShuffledRowRDD.scala | 2 +- .../adaptive/OptimizeSkewedPartitions.scala | 261 ++++++++++++++++++ .../execution/adaptive/QueryStageExec.scala | 3 +- .../adaptive/ReduceNumShufflePartitions.scala | 72 +++-- .../adaptive/SkewedShuffledRowRDD.scala | 90 ++++++ .../exchange/ShuffleExchangeExec.scala | 17 +- .../ReduceNumShufflePartitionsSuite.scala | 2 +- .../adaptive/AdaptiveQueryExecSuite.scala | 137 +++++++++ .../apache/spark/sql/test/SQLTestData.scala | 20 ++ 15 files changed, 755 insertions(+), 42 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index d2169246fe7e..03fdf82eab2d 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -355,6 +355,21 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] + /** + * Called from executors to get the server URIs and output sizes for each shuffle block that + * needs to be read from a specific map output partitions (partitionIndex) and is + * produced by a range mapper (startMapId, endMapId) + * + * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, + * and the second item is a sequence of (shuffle block id, shuffle block size, map index) + * tuples describing the shuffle blocks that are stored at that block manager. + */ + def getMapSizesByRangeMapIndex( + shuffleId: Int, + partitionIndex: Int, + startMapId: Int, + endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] + /** * Deletes map output status information for the specified shuffle stage. */ @@ -688,20 +703,25 @@ private[spark] class MapOutputTrackerMaster( } /** - * Return the location where the Mapper ran. The locations each includes both a host and an + * Return the locations where the Mappers ran. The locations each includes both a host and an * executor id on that host. * * @param dep shuffle dependency object - * @param mapId the map id + * @param startMapId the start map id + * @param endMapId the end map id * @return a sequence of locations where task runs. */ - def getMapLocation(dep: ShuffleDependency[_, _, _], mapId: Int): Seq[String] = + def getMapLocation( + dep: ShuffleDependency[_, _, _], + startMapId: Int, + endMapId: Int): Seq[String] = { val shuffleStatus = shuffleStatuses.get(dep.shuffleId).orNull if (shuffleStatus != null) { shuffleStatus.withMapStatuses { statuses => - if (mapId >= 0 && mapId < statuses.length) { - Seq(statuses(mapId).location.host) + if (startMapId < endMapId && (startMapId >= 0 && endMapId < statuses.length)) { + val statusesPicked = statuses.slice(startMapId, endMapId).filter(_ != null) + statusesPicked.map { status => status.location.host}.toSeq } else { Nil } @@ -766,6 +786,22 @@ private[spark] class MapOutputTrackerMaster( } } + override def getMapSizesByRangeMapIndex( + shuffleId: Int, + partitionIndex: Int, + startMapId: Int, + endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + shuffleStatuses.get(shuffleId) match { + case Some(shuffleStatus) => + shuffleStatus.withMapStatuses { statuses => + MapOutputTracker.convertMapStatuses( + shuffleId, partitionIndex, statuses, startMapId, endMapId) + } + case None => + Iterator.empty + } + } + override def stop(): Unit = { mapOutputRequests.offer(PoisonPill) threadpool.shutdown() @@ -830,6 +866,22 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } } + override def getMapSizesByRangeMapIndex( + shuffleId: Int, + partitionIndex: Int, + startMapId: Int, + endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + val statuses = getStatuses(shuffleId, conf) + try { + MapOutputTracker.convertMapStatuses(shuffleId, partitionIndex, statuses, startMapId, endMapId) + } catch { + case e: MetadataFetchFailedException => + // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: + mapStatuses.clear() + throw e + } + } + /** * Get or fetch the array of MapStatuses for a given shuffle ID. NOTE: clients MUST synchronize * on this array when reading it, because on the driver, we may be changing it in place. @@ -1012,4 +1064,50 @@ private[spark] object MapOutputTracker extends Logging { splitsByAddress.iterator } + + /** + * Given an array of map statuses, a specific map output partitions and a range + * mappers (startMapId, endMapId),returns a sequence that, for each block manager ID, + * lists the shuffle block IDs and corresponding shuffle + * block sizes stored at that block manager. + * Note that empty blocks are filtered in the result. + * + * If any of the statuses is null (indicating a missing location due to a failed mapper), + * throws a FetchFailedException. + * + * @param shuffleId Identifier for the shuffle + * @param partitionIndex Specific of map output partition ID + * @param statuses List of map statuses, indexed by map partition index. + * @param startMapId Start Map ID + * @param endMapId End map ID + * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, + * and the second item is a sequence of (shuffle block id, shuffle block size, map index) + * tuples describing the shuffle blocks that are stored at that block manager. + */ + def convertMapStatuses( + shuffleId: Int, + partitionIndex: Int, + statuses: Array[MapStatus], + startMapId: Int, + endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + assert (statuses != null) + val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] + val iter = statuses.iterator.zipWithIndex + for ((status, mapIndex) <- iter.slice(startMapId, endMapId)) { + if (status == null) { + val errorMessage = s"Missing an output location for shuffle $shuffleId" + logError(errorMessage) + throw new MetadataFetchFailedException(shuffleId, partitionIndex, errorMessage) + } else { + val size = status.getSizeForBlock(partitionIndex) + if (size != 0) { + splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += + ((ShuffleBlockId(shuffleId, status.mapId, partitionIndex), size, mapIndex)) + } + } + } + + splitsByAddress.iterator + } + } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 01aa43eb9763..fa05795f921c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -66,6 +66,18 @@ private[spark] trait ShuffleManager { context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] + /** + * Get a reader for the specific partitionIndex in map output statistics that are + * produced by range mappers. Called on executors by reduce tasks. + */ + def getReaderForRangeMapper[K, C]( + handle: ShuffleHandle, + partitionIndex: Int, + startMapId: Int, + endMapId: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] + /** * Remove a shuffle's metadata from the ShuffleManager. * @return true if the metadata removed successfully, otherwise false. diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 3cd04de0f741..675934decbd4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -145,6 +145,20 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) } + override def getReaderForRangeMapper[K, C]( + handle: ShuffleHandle, + partitionIndex: Int, + startMapId: Int, + endMapId: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByRangeMapIndex( + handle.shuffleId, partitionIndex, startMapId, endMapId) + new BlockStoreShuffleReader( + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, + shouldBatchFetch = canUseBatchFetch(partitionIndex, partitionIndex + 1, context)) + } + /** Get a writer for a given partition. Called on executors by map tasks. */ override def getWriter[K, V]( handle: ShuffleHandle, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1e05b6e2f99e..49e42eb53b72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -451,6 +451,34 @@ object SQLConf { .booleanConf .createWithDefault(true) + val ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED = buildConf("spark.sql.adaptive.skewedJoin.enabled") + .doc("When true and adaptive execution is enabled, a skewed join is automatically handled at " + + "runtime.") + .booleanConf + .createWithDefault(true) + + val ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR = + buildConf("spark.sql.adaptive.skewedPartitionFactor") + .doc("A partition is considered as a skewed partition if its size is larger than" + + " this factor multiple the median partition size and also larger than " + + "spark.sql.adaptive.skewedPartitionSizeThreshold.") + .intConf + .createWithDefault(10) + + val ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD = + buildConf("spark.sql.adaptive.skewedPartitionSizeThreshold") + .doc("Configures the minimum size in bytes for a partition that is considered as a skewed " + + "partition in adaptive skewed join.") + .longConf + .createWithDefault(64 * 1024 * 1024L) + + val ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS = + buildConf("spark.sql.adaptive.skewedPartitionMaxSplits") + .doc("Configures the maximum number of task to handle a skewed partition in adaptive skewed" + + "join.") + .intConf + .createWithDefault(5) + val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") .doc("The relation with a non-empty partition ratio lower than this config will not be " + @@ -2275,6 +2303,15 @@ class SQLConf extends Serializable with Logging { def maxNumPostShufflePartitions: Int = getConf(SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS).getOrElse(numShufflePartitions) + def adaptiveSkewedJoinEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED) + + def adaptiveSkewedFactor: Int = getConf(ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR) + + def adaptiveSkewedSizeThreshold: Long = + getConf(ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD) + + def adaptiveSkewedMaxSplits: Int = getConf(ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS) + def minBatchesToRetain: Int = getConf(MIN_BATCHES_TO_RETAIN) def maxBatchesToRetainInMemory: Int = getConf(MAX_BATCHES_TO_RETAIN_IN_MEMORY) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index 4c19f95796d0..202f68c5bb0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -116,7 +116,8 @@ class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: A class ShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], metrics: Map[String, SQLMetric], - specifiedPartitionStartIndices: Option[Array[Int]] = None) + specifiedPartitionStartIndices: Option[Array[Int]] = None, + specifiedPartitionEndIndices: Option[Array[Int]] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { if (SQLConf.get.fetchShuffleBlocksInBatchEnabled) { @@ -134,23 +135,24 @@ class ShuffledRowRDD( (0 until numPreShufflePartitions).toArray } - private[this] val part: Partitioner = - new CoalescedPartitioner(dependency.partitioner, partitionStartIndices) - override def getDependencies: Seq[Dependency[_]] = List(dependency) - override val partitioner: Option[Partitioner] = Some(part) + override val partitioner: Option[Partitioner] = specifiedPartitionEndIndices match { + case Some(indices) => None + case None => Some(new CoalescedPartitioner(dependency.partitioner, partitionStartIndices)) + } override def getPartitions: Array[Partition] = { - assert(partitionStartIndices.length == part.numPartitions) Array.tabulate[Partition](partitionStartIndices.length) { i => val startIndex = partitionStartIndices(i) - val endIndex = - if (i < partitionStartIndices.length - 1) { + val endIndex = specifiedPartitionEndIndices match { + case Some(indices) => indices(i) + case None => if (i < partitionStartIndices.length - 1) { partitionStartIndices(i + 1) } else { numPreShufflePartitions } + } new ShuffledRowRDDPartition(i, startIndex, endIndex) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index f5591072f696..42e18f2708c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -87,6 +87,10 @@ case class AdaptiveSparkPlanExec( // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( ReuseAdaptiveSubquery(conf, context.subqueryCache), + // Here the 'OptimizeSkewedPartitions' rule should be executed + // before 'ReduceNumShufflePartitions', as the skewed partition handled + // in 'OptimizeSkewedPartitions' rule, should be omitted in 'ReduceNumShufflePartitions'. + OptimizeSkewedPartitions(conf), ReduceNumShufflePartitions(conf), // The rule of 'OptimizeLocalShuffleReader' need to make use of the 'partitionStartIndices' // in 'ReduceNumShufflePartitions' rule. So it must be after 'ReduceNumShufflePartitions' rule. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala index 6385ea67c49f..faacc7e91ae1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala @@ -82,7 +82,7 @@ class LocalShuffledRowRDD( override def getPreferredLocations(partition: Partition): Seq[String] = { val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - tracker.getMapLocation(dependency, partition.index) + tracker.getMapLocation(dependency, partition.index, partition.index + 1) } override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala new file mode 100644 index 000000000000..ce4ef0df2236 --- /dev/null +++ b/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 + val numSplits = Math.min(conf.adaptiveSkewedMaxSplits, + Math.min(factor.toInt, numMappers)) + equallyDivide(numMappers, numSplits).toArray + } + + 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 + } else false + + joinTypeSupported && statisticsReady + } + + private def supportSplitOnLeftPartition(joinType: JoinType) = joinType != RightOuter + + private def supportSplitOnRightPartition(joinType: JoinType) = { + joinType != LeftOuter && joinType != LeftSemi && joinType != LeftAnti + } + + def handleSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { + case smj @ SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, + SortExec(_, _, left: QueryStageExec, _), + SortExec(_, _, right: QueryStageExec, _)) + if supportOptimization(joinType, left, right) => + val leftStats = getStatistics(left) + val rightStats = getStatistics(right) + val numPartitions = leftStats.bytesByPartitionId.length + + val leftMedSize = medianSize(leftStats) + val rightMedSize = medianSize(rightStats) + logInfo(s"HandlingSkewedJoin left medSize: ($leftMedSize)" + + s" right medSize ($rightMedSize)") + logInfo(s"left bytes Max : ${leftStats.bytesByPartitionId.max}") + logInfo(s"right bytes Max : ${rightStats.bytesByPartitionId.max}") + + val skewedPartitions = mutable.HashSet[Int]() + val subJoins = mutable.ArrayBuffer[SparkPlan](smj) + for (partitionId <- 0 until numPartitions) { + val isLeftSkew = isSkewed(leftStats, partitionId, leftMedSize) + val isRightSkew = isSkewed(rightStats, partitionId, rightMedSize) + val isSkewAndSupportsSplit = + (isLeftSkew && supportSplitOnLeftPartition(joinType)) || + (isRightSkew && supportSplitOnRightPartition(joinType)) + + if (isSkewAndSupportsSplit) { + skewedPartitions += partitionId + val leftMapIdStartIndices = if (isLeftSkew && supportSplitOnLeftPartition(joinType)) { + estimateMapIdStartIndices(left, partitionId, leftMedSize) + } else { + Array(0) + } + val rightMapIdStartIndices = if (isRightSkew && supportSplitOnRightPartition(joinType)) { + estimateMapIdStartIndices(right, partitionId, rightMedSize) + } else { + Array(0) + } + + for (i <- 0 until leftMapIdStartIndices.length; + j <- 0 until rightMapIdStartIndices.length) { + val leftEndMapId = if (i == leftMapIdStartIndices.length - 1) { + getShuffleStage(left).plan.shuffleDependency.rdd.partitions.length + } else { + leftMapIdStartIndices(i + 1) + } + val rightEndMapId = if (j == rightMapIdStartIndices.length - 1) { + getShuffleStage(right). + plan.shuffleDependency.rdd.partitions.length + } else { + rightMapIdStartIndices(j + 1) + } + // For the skewed partition, we set the id of shuffle query stage to -1. + // And skip this shuffle query stage optimization in 'ReduceNumShufflePartitions' rule. + val leftSkewedReader = + SkewedShuffleReaderExec(getShuffleStage(left).copy(id = -1), + partitionId, leftMapIdStartIndices(i), leftEndMapId) + + val rightSkewedReader = + SkewedShuffleReaderExec(getShuffleStage(right).copy(id = -1), + partitionId, rightMapIdStartIndices(j), rightEndMapId) + + subJoins += + SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, + leftSkewedReader, rightSkewedReader) + } + } + } + logInfo(s"skewed partition number is ${skewedPartitions.size}") + if (skewedPartitions.size > 0) { + getShuffleStage(left).skewedPartitions = skewedPartitions + getShuffleStage(right).skewedPartitions = skewedPartitions + UnionExec(subJoins.toList) + } else { + smj + } + } + + override def apply(plan: SparkPlan): SparkPlan = { + if (!conf.adaptiveSkewedJoinEnabled) { + return plan + } + + def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { + case _: LocalShuffleReaderExec => Nil + case _: SkewedShuffleReaderExec => Nil + case stage: ShuffleQueryStageExec => Seq(stage) + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => Seq(stage) + case _ => plan.children.flatMap(collectShuffleStages) + } + + val shuffleStages = collectShuffleStages(plan) + + if (shuffleStages.length == 2) { + // Currently we only support handling skewed join for 2 table join. + handleSkewJoin(plan) + } else { + plan + } + } +} + +case class SkewedShuffleReaderExec( + child: QueryStageExec, + partitionIndex: Int, + startMapId: Int, + endMapId: Int) extends UnaryExecNode { + + override def output: Seq[Attribute] = child.output + + override def doCanonicalize(): SparkPlan = child.canonicalized + + override def outputPartitioning: Partitioning = { + UnknownPartitioning(1) + } + private var cachedSkewedShuffleRDD: SkewedShuffledRowRDD = null + + override def doExecute(): RDD[InternalRow] = { + if (cachedSkewedShuffleRDD == null) { + cachedSkewedShuffleRDD = child match { + case stage: ShuffleQueryStageExec => + stage.plan.createSkewedShuffleRDD(partitionIndex, startMapId, endMapId) + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => + stage.plan.createSkewedShuffleRDD(partitionIndex, startMapId, endMapId) + } + } + cachedSkewedShuffleRDD + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index d5dc1be63f06..6ba799af2888 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -134,7 +134,8 @@ abstract class QueryStageExec extends LeafExecNode { */ case class ShuffleQueryStageExec( override val id: Int, - override val plan: SparkPlan) extends QueryStageExec { + override val plan: SparkPlan, + var skewedPartitions: mutable.HashSet[Int] = mutable.HashSet.empty) extends QueryStageExec { @transient val shuffle = plan match { case s: ShuffleExchangeExec => s diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 3767e6cae46e..3f4ed12c195c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.adaptive +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.MapOutputStatistics @@ -63,6 +64,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil + case _: SkewedShuffleReaderExec => Nil case stage: ShuffleQueryStageExec => Seq(stage) case _ => plan.children.flatMap(collectShuffleStages) } @@ -87,14 +89,16 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { val distinctNumPreShufflePartitions = validMetrics.map(stats => stats.bytesByPartitionId.length).distinct if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { - val partitionStartIndices = estimatePartitionStartIndices(validMetrics.toArray) + val omittedPartitions = shuffleStages(0).skewedPartitions + val (partitionStartIndices, partitionEndIndices) = estimatePartitionStartIndices( + validMetrics.toArray, omittedPartitions) // This transformation adds new nodes, so we must use `transformUp` here. plan.transformUp { // even for shuffle exchange whose input RDD has 0 partition, we should still update its // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same // number of output partitions. case stage: ShuffleQueryStageExec => - CoalescedShuffleReaderExec(stage, partitionStartIndices) + CoalescedShuffleReaderExec(stage, partitionStartIndices, partitionEndIndices) } } else { plan @@ -103,12 +107,14 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { } /** - * Estimates partition start indices for post-shuffle partitions based on - * mapOutputStatistics provided by all pre-shuffle stages. + * Estimates partition start and end indices for post-shuffle partitions based on + * mapOutputStatistics provided by all pre-shuffle stages and skip the omittedPartitions + * already handled in skewed partition optimization. */ // visible for testing. private[sql] def estimatePartitionStartIndices( - mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = { + mapOutputStatistics: Array[MapOutputStatistics], + omittedPartitions: mutable.HashSet[Int] = mutable.HashSet.empty): (Array[Int], Array[Int]) = { val minNumPostShufflePartitions = conf.minNumPostShufflePartitions val advisoryTargetPostShuffleInputSize = conf.targetPostShuffleInputSize // If minNumPostShufflePartitions is defined, it is possible that we need to use a @@ -144,36 +150,57 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { val numPreShufflePartitions = distinctNumPreShufflePartitions.head val partitionStartIndices = ArrayBuffer[Int]() - // The first element of partitionStartIndices is always 0. - partitionStartIndices += 0 + val partitionEndIndices = ArrayBuffer[Int]() - var postShuffleInputSize = 0L + def nextStartIndex(i: Int): Int = { + var index = i + while (index < numPreShufflePartitions && omittedPartitions.contains(index)) { + index = index + 1 + } + index + } - var i = 0 - while (i < numPreShufflePartitions) { - // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages. - // Then, we add the total size to postShuffleInputSize. - var nextShuffleInputSize = 0L + def partitionSize(partitionId: Int): Long = { + var size = 0L var j = 0 while (j < mapOutputStatistics.length) { - nextShuffleInputSize += mapOutputStatistics(j).bytesByPartitionId(i) + val statistics = mapOutputStatistics(j) + size += statistics.bytesByPartitionId(partitionId) j += 1 } + size + } + + val firstStartIndex = nextStartIndex(0) + + partitionStartIndices += firstStartIndex + + var postShuffleInputSize = partitionSize(firstStartIndex) + + var i = firstStartIndex + var nextIndex = nextStartIndex(i + 1) + while (nextIndex < numPreShufflePartitions) { + // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages. + // Then, we add the total size to postShuffleInputSize. + var nextShuffleInputSize = partitionSize(nextIndex) // If including the nextShuffleInputSize would exceed the target partition size, then start a // new partition. - if (i > 0 && postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize) { - partitionStartIndices += i + if (nextIndex != i + 1 || + (postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize)) { + partitionEndIndices += i + 1 + partitionStartIndices += nextIndex // reset postShuffleInputSize. postShuffleInputSize = nextShuffleInputSize + i = nextIndex } else { postShuffleInputSize += nextShuffleInputSize + i += 1 } - - i += 1 + nextIndex = nextStartIndex(nextIndex + 1) } - - partitionStartIndices.toArray + partitionEndIndices += i + 1 + (partitionStartIndices.toArray, partitionEndIndices.toArray) } } @@ -186,7 +213,8 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { */ case class CoalescedShuffleReaderExec( child: SparkPlan, - partitionStartIndices: Array[Int]) extends UnaryExecNode { + partitionStartIndices: Array[Int], + partitionEndIndices: Array[Int]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output @@ -200,7 +228,7 @@ case class CoalescedShuffleReaderExec( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.shuffle.createShuffledRDD(Some(partitionStartIndices)) + stage.shuffle.createShuffledRDD(Some(partitionStartIndices), Some(partitionEndIndices)) case _ => throw new IllegalStateException("operating on canonicalization plan") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala new file mode 100644 index 000000000000..a707c847ca8d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala @@ -0,0 +1,90 @@ +/* + * 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._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} + +/** + * The [[Partition]] used by [[SkewedShuffledRowRDD]]. A post-shuffle partition + * (identified by `postShufflePartitionIndex`) contains a range of pre-shuffle partitions + * (`preShufflePartitionIndex` from `startMapId` to `endMapId - 1`, inclusive). + */ +private final class SkewedShuffledRowRDDPartition( + val postShufflePartitionIndex: Int, + val preShufflePartitionIndex: Int, + val startMapId: Int, + val endMapId: Int) extends Partition{ + override val index: Int = postShufflePartitionIndex +} + +/** + * This is a specialized version of [[org.apache.spark.sql.execution.ShuffledRowRDD]]. This is used + * in Spark SQL adaptive execution to solve data skew issues. This RDD includes rearranged + * partitions from mappers. + * + * This RDD takes a [[ShuffleDependency]] (`dependency`), a partitionIndex + * and the range of startMapId to endMapId. + * + */ +class SkewedShuffledRowRDD( + var dependency: ShuffleDependency[Int, InternalRow, InternalRow], + partitionIndex: Int, + startMapId: Int, + endMapId: Int, + metrics: Map[String, SQLMetric]) + extends RDD[InternalRow](dependency.rdd.context, Nil) { + + override def getDependencies: Seq[Dependency[_]] = List(dependency) + override def getPartitions: Array[Partition] = { + Array.tabulate[Partition](1) { i => + new SkewedShuffledRowRDDPartition(i, partitionIndex, startMapId, endMapId) + } + } + + override def getPreferredLocations(partition: Partition): Seq[String] = { + val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + val skewedPartition = partition.asInstanceOf[SkewedShuffledRowRDDPartition] + tracker.getMapLocation(dependency, skewedPartition.startMapId, skewedPartition.endMapId) + } + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val skewedPartition = split.asInstanceOf[SkewedShuffledRowRDDPartition] + + val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() + // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, + // as well as the `tempMetrics` for basic shuffle metrics. + val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) + + val reader = SparkEnv.get.shuffleManager.getReaderForRangeMapper( + dependency.shuffleHandle, + skewedPartition.preShufflePartitionIndex, + skewedPartition.startMapId, + skewedPartition.endMapId, + context, + sqlMetricsReporter) + reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) + } + + override def clearDependencies() { + super.clearDependencies() + dependency = null + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 4281f01e2756..8ab25449603d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Uns import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.LocalShuffledRowRDD +import org.apache.spark.sql.execution.adaptive.{LocalShuffledRowRDD, SkewedShuffledRowRDD} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -90,8 +90,10 @@ case class ShuffleExchangeExec( writeMetrics) } - def createShuffledRDD(partitionStartIndices: Option[Array[Int]]): ShuffledRowRDD = { - new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices) + def createShuffledRDD( + partitionStartIndices: Option[Array[Int]], + partitionEndIndices: Option[Array[Int]]): ShuffledRowRDD = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices, partitionEndIndices) } def createLocalShuffleRDD( @@ -99,6 +101,13 @@ case class ShuffleExchangeExec( new LocalShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndicesPerMapper) } + def createSkewedShuffleRDD( + partitionIndex: Int, + startMapId: Int, + endMapId: Int): SkewedShuffledRowRDD = { + new SkewedShuffledRowRDD(shuffleDependency, partitionIndex, startMapId, endMapId, readMetrics) + } + /** * Caches the created ShuffleRowRDD so we can reuse that. */ @@ -107,7 +116,7 @@ case class ShuffleExchangeExec( protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { // Returns the same ShuffleRowRDD if this plan is used by multiple plans. if (cachedShuffleRDD == null) { - cachedShuffleRDD = createShuffledRDD(None) + cachedShuffleRDD = createShuffledRDD(None, None) } cachedShuffleRDD } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 70d5a8a615ad..026fd726cc99 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -61,7 +61,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA new MapOutputStatistics(index, bytesByPartitionId) } val estimatedPartitionStartIndices = - rule.estimatePartitionStartIndices(mapOutputStatistics) + rule.estimatePartitionStartIndices(mapOutputStatistics)._1 assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index fb24eaf2a4bf..4d5eb73bfe86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -558,6 +558,7 @@ class AdaptiveQueryExecSuite } } } +<<<<<<< HEAD test("SPARK-29906: AQE should not introduce extra shuffle for outermost limit") { var numStages = 0 @@ -576,6 +577,142 @@ class AdaptiveQueryExecSuite } } finally { spark.sparkContext.removeSparkListener(listener) +======= + test("adaptive skew join both in left and right for inner join ") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR.key -> "1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 join skewData2 ON key1 = key2") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + // left stats: [4403, 0, 1927, 1927, 1927] + // right stats:[6292, 0, 0, 0, 0] + // the partition 0 in both left and right are all skewed. + // And the partition 0 in left is split to 2 smj and the partition 0 + // in right is split to 5 smjs. So total 11 smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) + assert(smjAfter.size == 11) + } + } + + test("adaptive skew join both in left and right for left outer join ") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR.key -> "1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + // left stats: [4403, 0, 1927, 1927, 1927] + // right stats:[6292, 0, 0, 0, 0] + // the partition 0 in both left and right are all skewed. + // But for left outer join, we don't split the right partition even skewed. + // And the partition 0 in left is split to 2 smj. So total 3 smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) + assert(smjAfter.size == 3) + } + } + test("adaptive skew join both in left and right for right outer join ") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR.key -> "1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + // left stats: [4403, 0, 1927, 1927, 1927] + // right stats:[6292, 0, 0, 0, 0] + // the partition 0 in both left and right are all skewed. + // But for right outer join, we don't split the left partition even skewed. + // And the partition 0 in right is split to 5 smj. So total 6 smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) + assert(smjAfter.size == 6) +>>>>>>> optimize skewed partition based on data size } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index c51faaf10f5d..85f943aa48c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -287,6 +287,24 @@ private[sql] trait SQLTestData { self => df } + protected lazy val skewData1: DataFrame = { + val df1 = + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 5 as key1", "id as value1").toDF() + df1.createOrReplaceTempView("skewData1") + df1 + } + + protected lazy val skewData2: DataFrame = { + val df2 = + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2").toDF() + df2.createOrReplaceTempView("skewData2") + df2 + } + /** * Initialize all test data such that all temp tables are properly registered. */ @@ -316,6 +334,8 @@ private[sql] trait SQLTestData { self => salary complexData courseSales + skewData1 + skewData2 } } From 84c0935355e8acba6a7977472f4b8a7a91e82142 Mon Sep 17 00:00:00 2001 From: jiake Date: Fri, 8 Nov 2019 14:52:50 +0800 Subject: [PATCH 02/23] code style fix --- .../sql/execution/adaptive/OptimizeSkewedPartitions.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index ce4ef0df2236..f51048a5fd54 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -107,10 +107,10 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } /** - * 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. - */ + * 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, From dd30240e6ba621e761ca2bd6170011967d2f8d9d Mon Sep 17 00:00:00 2001 From: jiake Date: Fri, 8 Nov 2019 15:39:07 +0800 Subject: [PATCH 03/23] import mutable package to fix compile error --- .../org/apache/spark/sql/execution/adaptive/QueryStageExec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 6ba799af2888..5afdcffe59d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.adaptive +import scala.collection.mutable import scala.concurrent.Future import org.apache.spark.{FutureAction, MapOutputStatistics} From 84d5f2f83bbd41457e91104aa386b05acb598516 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 12 Nov 2019 16:45:23 +0800 Subject: [PATCH 04/23] merge the range reader and split the mappers based on data size --- .../org/apache/spark/MapOutputTracker.scala | 103 +++++------------- .../apache/spark/shuffle/ShuffleManager.scala | 22 +--- .../shuffle/sort/SortShuffleManager.scala | 22 +--- .../apache/spark/sql/internal/SQLConf.scala | 9 -- .../adaptive/LocalShuffledRowRDD.scala | 4 +- .../adaptive/OptimizeSkewedPartitions.scala | 68 ++++++++---- .../adaptive/ReduceNumShufflePartitions.scala | 2 +- .../adaptive/SkewedShuffledRowRDD.scala | 5 +- .../adaptive/AdaptiveQueryExecSuite.scala | 61 +++++++++-- .../apache/spark/sql/test/SQLTestData.scala | 20 ---- 10 files changed, 140 insertions(+), 176 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 03fdf82eab2d..8178a653ae01 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -343,30 +343,17 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** * Called from executors to get the server URIs and output sizes for each shuffle block that * needs to be read from a given range of map output partitions (startPartition is included but - * endPartition is excluded from the range) and is produced by a specific mapper. + * endPartition is excluded from the range) and is produced by + * a range mapper (startMapId, endMapId, startMapId is included and the endMapId is excluded). * * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. */ - def getMapSizesByMapIndex( + def getMapSizesByRange( shuffleId: Int, - mapIndex: Int, startPartition: Int, - endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] - - /** - * Called from executors to get the server URIs and output sizes for each shuffle block that - * needs to be read from a specific map output partitions (partitionIndex) and is - * produced by a range mapper (startMapId, endMapId) - * - * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, - * and the second item is a sequence of (shuffle block id, shuffle block size, map index) - * tuples describing the shuffle blocks that are stored at that block manager. - */ - def getMapSizesByRangeMapIndex( - shuffleId: Int, - partitionIndex: Int, + endPartition: Int, startMapId: Int, endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] @@ -764,44 +751,25 @@ private[spark] class MapOutputTrackerMaster( } } - override def getMapSizesByMapIndex( - shuffleId: Int, - mapIndex: Int, - startPartition: Int, - endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, mapIndex $mapIndex" + - s"partitions $startPartition-$endPartition") - shuffleStatuses.get(shuffleId) match { - case Some (shuffleStatus) => - shuffleStatus.withMapStatuses { statuses => - MapOutputTracker.convertMapStatuses( - shuffleId, - startPartition, - endPartition, - statuses, - Some(mapIndex)) - } - case None => - Iterator.empty - } - } - - override def getMapSizesByRangeMapIndex( - shuffleId: Int, - partitionIndex: Int, - startMapId: Int, - endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + override def getMapSizesByRange( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + startMapId: Int, + endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => shuffleStatus.withMapStatuses { statuses => MapOutputTracker.convertMapStatuses( - shuffleId, partitionIndex, statuses, startMapId, endMapId) + shuffleId, startPartition, endPartition, statuses, startMapId, endMapId) } case None => Iterator.empty } } + + override def stop(): Unit = { mapOutputRequests.offer(PoisonPill) threadpool.shutdown() @@ -847,33 +815,16 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } } - override def getMapSizesByMapIndex( + override def getMapSizesByRange( shuffleId: Int, - mapIndex: Int, startPartition: Int, - endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, mapIndex $mapIndex" + - s"partitions $startPartition-$endPartition") - val statuses = getStatuses(shuffleId, conf) - try { - MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, - statuses, Some(mapIndex)) - } catch { - case e: MetadataFetchFailedException => - // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: - mapStatuses.clear() - throw e - } - } - - override def getMapSizesByRangeMapIndex( - shuffleId: Int, - partitionIndex: Int, + endPartition: Int, startMapId: Int, endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { val statuses = getStatuses(shuffleId, conf) try { - MapOutputTracker.convertMapStatuses(shuffleId, partitionIndex, statuses, startMapId, endMapId) + MapOutputTracker.convertMapStatuses( + shuffleId, startPartition, endPartition, statuses, startMapId, endMapId) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -1066,7 +1017,7 @@ private[spark] object MapOutputTracker extends Logging { } /** - * Given an array of map statuses, a specific map output partitions and a range + * Given an array of map statuses, a range map output partitions and a range * mappers (startMapId, endMapId),returns a sequence that, for each block manager ID, * lists the shuffle block IDs and corresponding shuffle * block sizes stored at that block manager. @@ -1076,7 +1027,8 @@ private[spark] object MapOutputTracker extends Logging { * throws a FetchFailedException. * * @param shuffleId Identifier for the shuffle - * @param partitionIndex Specific of map output partition ID + * @param startPartition Start map output partition ID + * @param endPartition End map output partition ID * @param statuses List of map statuses, indexed by map partition index. * @param startMapId Start Map ID * @param endMapId End map ID @@ -1086,7 +1038,8 @@ private[spark] object MapOutputTracker extends Logging { */ def convertMapStatuses( shuffleId: Int, - partitionIndex: Int, + startPartition: Int, + endPartition: Int, statuses: Array[MapStatus], startMapId: Int, endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { @@ -1097,12 +1050,14 @@ private[spark] object MapOutputTracker extends Logging { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" logError(errorMessage) - throw new MetadataFetchFailedException(shuffleId, partitionIndex, errorMessage) + throw new MetadataFetchFailedException(shuffleId, startPartition, errorMessage) } else { - val size = status.getSizeForBlock(partitionIndex) - if (size != 0) { - splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, status.mapId, partitionIndex), size, mapIndex)) + for (part <- startPartition until endPartition) { + val size = status.getSizeForBlock(part) + if (size != 0) { + splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += + ((ShuffleBlockId(shuffleId, status.mapId, part), size, mapIndex)) + } } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index fa05795f921c..bbeb10f6de49 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -55,26 +55,16 @@ private[spark] trait ShuffleManager { metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] /** - * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) - * that are produced by one specific mapper. Called on executors by reduce tasks. - */ - def getReaderForOneMapper[K, C]( - handle: ShuffleHandle, - mapIndex: Int, - startPartition: Int, - endPartition: Int, - context: TaskContext, - metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] - - /** - * Get a reader for the specific partitionIndex in map output statistics that are - * produced by range mappers. Called on executors by reduce tasks. + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to + * read from map output (startMapId to endMapId - 1, inclusive). + * Called on executors by reduce tasks. */ - def getReaderForRangeMapper[K, C]( + def getReaderForRange[K, C]( handle: ShuffleHandle, - partitionIndex: Int, startMapId: Int, endMapId: Int, + startPartition: Int, + endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 675934decbd4..0a2a862f2ddd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -131,32 +131,20 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) } - override def getReaderForOneMapper[K, C]( + override def getReaderForRange[K, C]( handle: ShuffleHandle, - mapIndex: Int, + startMapId: Int, + endMapId: Int, startPartition: Int, endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByMapIndex( - handle.shuffleId, mapIndex, startPartition, endPartition) + val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByRange( + handle.shuffleId, startPartition, endPartition, startMapId, endMapId) new BlockStoreShuffleReader( handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) - } - override def getReaderForRangeMapper[K, C]( - handle: ShuffleHandle, - partitionIndex: Int, - startMapId: Int, - endMapId: Int, - context: TaskContext, - metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByRangeMapIndex( - handle.shuffleId, partitionIndex, startMapId, endMapId) - new BlockStoreShuffleReader( - handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, - shouldBatchFetch = canUseBatchFetch(partitionIndex, partitionIndex + 1, context)) } /** Get a writer for a given partition. Called on executors by map tasks. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 49e42eb53b72..f89b045c487f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -472,13 +472,6 @@ object SQLConf { .longConf .createWithDefault(64 * 1024 * 1024L) - val ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS = - buildConf("spark.sql.adaptive.skewedPartitionMaxSplits") - .doc("Configures the maximum number of task to handle a skewed partition in adaptive skewed" + - "join.") - .intConf - .createWithDefault(5) - val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") .doc("The relation with a non-empty partition ratio lower than this config will not be " + @@ -2310,8 +2303,6 @@ class SQLConf extends Serializable with Logging { def adaptiveSkewedSizeThreshold: Long = getConf(ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD) - def adaptiveSkewedMaxSplits: Int = getConf(ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS) - def minBatchesToRetain: Int = getConf(MIN_BATCHES_TO_RETAIN) def maxBatchesToRetainInMemory: Int = getConf(MAX_BATCHES_TO_RETAIN_IN_MEMORY) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala index faacc7e91ae1..19b78f5e36c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala @@ -92,9 +92,11 @@ class LocalShuffledRowRDD( // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, // as well as the `tempMetrics` for basic shuffle metrics. val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) - val reader = SparkEnv.get.shuffleManager.getReaderForOneMapper( + + val reader = SparkEnv.get.shuffleManager.getReaderForRange( dependency.shuffleHandle, mapIndex, + mapIndex + 1, localRowPartition.startPartition, localRowPartition.endPartition, context, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index f51048a5fd54..fcc074f81c36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -18,9 +18,10 @@ 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 +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 @@ -57,17 +58,38 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { 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) + /* + * 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 } /** @@ -79,14 +101,11 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { 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 - val numSplits = Math.min(conf.adaptiveSkewedMaxSplits, - Math.min(factor.toInt, numMappers)) - equallyDivide(numMappers, numSplits).toArray + 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 = { @@ -185,11 +204,11 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { // For the skewed partition, we set the id of shuffle query stage to -1. // And skip this shuffle query stage optimization in 'ReduceNumShufflePartitions' rule. val leftSkewedReader = - SkewedShuffleReaderExec(getShuffleStage(left).copy(id = -1), + PostShufflePartitionReader(getShuffleStage(left).copy(id = -1), partitionId, leftMapIdStartIndices(i), leftEndMapId) val rightSkewedReader = - SkewedShuffleReaderExec(getShuffleStage(right).copy(id = -1), + PostShufflePartitionReader(getShuffleStage(right).copy(id = -1), partitionId, rightMapIdStartIndices(j), rightEndMapId) subJoins += @@ -215,7 +234,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil - case _: SkewedShuffleReaderExec => Nil + case _: PostShufflePartitionReader => Nil case stage: ShuffleQueryStageExec => Seq(stage) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => Seq(stage) case _ => plan.children.flatMap(collectShuffleStages) @@ -228,11 +247,12 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { handleSkewJoin(plan) } else { plan + } } } -case class SkewedShuffleReaderExec( +case class PostShufflePartitionReader( child: QueryStageExec, partitionIndex: Int, startMapId: Int, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 3f4ed12c195c..b93d72e6e03c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -64,7 +64,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil - case _: SkewedShuffleReaderExec => Nil + case _: PostShufflePartitionReader => Nil case stage: ShuffleQueryStageExec => Seq(stage) case _ => plan.children.flatMap(collectShuffleStages) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala index a707c847ca8d..d94d28092509 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala @@ -73,11 +73,12 @@ class SkewedShuffledRowRDD( // as well as the `tempMetrics` for basic shuffle metrics. val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) - val reader = SparkEnv.get.shuffleManager.getReaderForRangeMapper( + val reader = SparkEnv.get.shuffleManager.getReaderForRange( dependency.shuffleHandle, - skewedPartition.preShufflePartitionIndex, skewedPartition.startMapId, skewedPartition.endMapId, + skewedPartition.preShufflePartitionIndex, + skewedPartition.preShufflePartitionIndex + 1, context, sqlMetricsReporter) reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 4d5eb73bfe86..0ee6d689b7b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -21,6 +21,7 @@ import java.io.File import java.net.URI import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec} @@ -37,6 +38,32 @@ class AdaptiveQueryExecSuite import testImplicits._ + protected lazy val skewData1: DataFrame = { + val df1 = + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 5 as key1", "id as value1").toDF() + df1.createOrReplaceTempView("skewData1") + df1 + } + + protected lazy val skewData2: DataFrame = { + val df2 = + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2").toDF() + df2.createOrReplaceTempView("skewData2") + df2 + } + + protected override def beforeAll(): Unit = { + super.beforeAll() + skewData1 + skewData2 + } + + + setupTestData() private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { @@ -558,7 +585,6 @@ class AdaptiveQueryExecSuite } } } -<<<<<<< HEAD test("SPARK-29906: AQE should not introduce extra shuffle for outermost limit") { var numStages = 0 @@ -577,13 +603,16 @@ class AdaptiveQueryExecSuite } } finally { spark.sparkContext.removeSparkListener(listener) -======= + } + } + test("adaptive skew join both in left and right for inner join ") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR.key -> "1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100") { + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "2000") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM skewData1 join skewData2 ON key1 = key2") val smj = findTopLevelSortMergeJoin(plan) @@ -591,8 +620,11 @@ class AdaptiveQueryExecSuite // left stats: [4403, 0, 1927, 1927, 1927] // right stats:[6292, 0, 0, 0, 0] // the partition 0 in both left and right are all skewed. - // And the partition 0 in left is split to 2 smj and the partition 0 - // in right is split to 5 smjs. So total 11 smjs. + // And the map data size of partition 0 in left is + // [539, 539, 490, 405, 405, 405, 405, 405, 405, 405]. So split to 3 smjs [0, 4, 8]. + // and the the map data size of partition 0 in right is + // [955, 593, 593, 593, 593, 593, 593, 593, 593, 593]. So split to 4 smjs [0, 2, 5, 8] + // So total 13 (3 x 4 + 1) smjs. // Union // +- SortMergeJoin // +- Sort @@ -620,7 +652,7 @@ class AdaptiveQueryExecSuite // +- ShuffleQueryStage val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 11) + assert(smjAfter.size == 13) } } @@ -629,7 +661,8 @@ class AdaptiveQueryExecSuite SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR.key -> "1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100") { + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "2000") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") val smj = findTopLevelSortMergeJoin(plan) @@ -638,7 +671,9 @@ class AdaptiveQueryExecSuite // right stats:[6292, 0, 0, 0, 0] // the partition 0 in both left and right are all skewed. // But for left outer join, we don't split the right partition even skewed. - // And the partition 0 in left is split to 2 smj. So total 3 smjs. + // And the map data size of partition 0 in left is + // [539, 539, 490, 405, 405, 405, 405, 405, 405, 405]. So split to 3 smjs [0, 4, 8]. + // So total 4 smjs. // Union // +- SortMergeJoin // +- Sort @@ -666,7 +701,7 @@ class AdaptiveQueryExecSuite // +- ShuffleQueryStage val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 3) + assert(smjAfter.size == 4) } } test("adaptive skew join both in left and right for right outer join ") { @@ -674,7 +709,8 @@ class AdaptiveQueryExecSuite SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR.key -> "1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100") { + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "2000") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") val smj = findTopLevelSortMergeJoin(plan) @@ -683,7 +719,9 @@ class AdaptiveQueryExecSuite // right stats:[6292, 0, 0, 0, 0] // the partition 0 in both left and right are all skewed. // But for right outer join, we don't split the left partition even skewed. - // And the partition 0 in right is split to 5 smj. So total 6 smjs. + // And the the map data size of partition 0 in right is + // [955, 593, 593, 593, 593, 593, 593, 593, 593, 593]. So split to 4 smjs [0, 2, 5, 8] + // So total 5 smjs. // Union // +- SortMergeJoin // +- Sort @@ -712,7 +750,6 @@ class AdaptiveQueryExecSuite val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) assert(smjAfter.size == 6) ->>>>>>> optimize skewed partition based on data size } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 85f943aa48c1..c51faaf10f5d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -287,24 +287,6 @@ private[sql] trait SQLTestData { self => df } - protected lazy val skewData1: DataFrame = { - val df1 = - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 5 as key1", "id as value1").toDF() - df1.createOrReplaceTempView("skewData1") - df1 - } - - protected lazy val skewData2: DataFrame = { - val df2 = - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 1 as key2", "id as value2").toDF() - df2.createOrReplaceTempView("skewData2") - df2 - } - /** * Initialize all test data such that all temp tables are properly registered. */ @@ -334,8 +316,6 @@ private[sql] trait SQLTestData { self => salary complexData courseSales - skewData1 - skewData2 } } From 782ee146643534fe6dd02e6c0f57d4a98bddaf72 Mon Sep 17 00:00:00 2001 From: jiake Date: Wed, 13 Nov 2019 11:07:36 +0800 Subject: [PATCH 05/23] merge the two convertMapStatuses method --- .../org/apache/spark/MapOutputTracker.scala | 57 ++----------------- 1 file changed, 5 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 8178a653ae01..7a54c7a00e98 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -744,7 +744,7 @@ private[spark] class MapOutputTrackerMaster( case Some (shuffleStatus) => shuffleStatus.withMapStatuses { statuses => MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses) + shuffleId, startPartition, endPartition, statuses, 0, shuffleStatus.mapStatuses.length) } case None => Iterator.empty @@ -806,7 +806,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr val statuses = getStatuses(shuffleId, conf) try { MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses) + shuffleId, startPartition, endPartition, statuses, 0, statuses.length) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -983,7 +983,8 @@ private[spark] object MapOutputTracker extends Logging { * @param startPartition Start of map output partition ID range (included in range) * @param endPartition End of map output partition ID range (excluded from range) * @param statuses List of map statuses, indexed by map partition index. - * @param mapIndex When specified, only shuffle blocks from this mapper will be processed. + * @param startMapId Start Map ID. + * @param endMapId End Map ID. * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. @@ -993,55 +994,7 @@ private[spark] object MapOutputTracker extends Logging { startPartition: Int, endPartition: Int, statuses: Array[MapStatus], - mapIndex : Option[Int] = None): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - assert (statuses != null) - val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] - val iter = statuses.iterator.zipWithIndex - for ((status, mapIndex) <- mapIndex.map(index => iter.filter(_._2 == index)).getOrElse(iter)) { - if (status == null) { - val errorMessage = s"Missing an output location for shuffle $shuffleId" - logError(errorMessage) - throw new MetadataFetchFailedException(shuffleId, startPartition, errorMessage) - } else { - for (part <- startPartition until endPartition) { - val size = status.getSizeForBlock(part) - if (size != 0) { - splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, status.mapId, part), size, mapIndex)) - } - } - } - } - - splitsByAddress.iterator - } - - /** - * Given an array of map statuses, a range map output partitions and a range - * mappers (startMapId, endMapId),returns a sequence that, for each block manager ID, - * lists the shuffle block IDs and corresponding shuffle - * block sizes stored at that block manager. - * Note that empty blocks are filtered in the result. - * - * If any of the statuses is null (indicating a missing location due to a failed mapper), - * throws a FetchFailedException. - * - * @param shuffleId Identifier for the shuffle - * @param startPartition Start map output partition ID - * @param endPartition End map output partition ID - * @param statuses List of map statuses, indexed by map partition index. - * @param startMapId Start Map ID - * @param endMapId End map ID - * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, - * and the second item is a sequence of (shuffle block id, shuffle block size, map index) - * tuples describing the shuffle blocks that are stored at that block manager. - */ - def convertMapStatuses( - shuffleId: Int, - startPartition: Int, - endPartition: Int, - statuses: Array[MapStatus], - startMapId: Int, + startMapId : Int, endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { assert (statuses != null) val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] From 98315ff571ad874b3457292c9088c4ca22bbdc51 Mon Sep 17 00:00:00 2001 From: jiake Date: Mon, 2 Dec 2019 16:36:24 +0800 Subject: [PATCH 06/23] change the type to Option[Array[(Int, Int)]] in ShuffledRowRDD --- .../spark/sql/execution/ShuffledRowRDD.scala | 13 ++++++------- .../adaptive/ReduceNumShufflePartitions.scala | 15 +++++++-------- .../execution/exchange/ShuffleExchangeExec.scala | 7 +++---- .../ReduceNumShufflePartitionsSuite.scala | 2 +- .../adaptive/AdaptiveQueryExecSuite.scala | 2 -- 5 files changed, 17 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index 202f68c5bb0e..8cae1b5ba0e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -116,8 +116,7 @@ class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: A class ShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], metrics: Map[String, SQLMetric], - specifiedPartitionStartIndices: Option[Array[Int]] = None, - specifiedPartitionEndIndices: Option[Array[Int]] = None) + specifiedPartitionIndices: Option[Array[(Int, Int)]] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { if (SQLConf.get.fetchShuffleBlocksInBatchEnabled) { @@ -127,8 +126,8 @@ class ShuffledRowRDD( private[this] val numPreShufflePartitions = dependency.partitioner.numPartitions - private[this] val partitionStartIndices: Array[Int] = specifiedPartitionStartIndices match { - case Some(indices) => indices + private[this] val partitionStartIndices: Array[Int] = specifiedPartitionIndices match { + case Some(indices) => indices.unzip._1 case None => // When specifiedPartitionStartIndices is not defined, every post-shuffle partition // corresponds to a pre-shuffle partition. @@ -137,7 +136,7 @@ class ShuffledRowRDD( override def getDependencies: Seq[Dependency[_]] = List(dependency) - override val partitioner: Option[Partitioner] = specifiedPartitionEndIndices match { + override val partitioner: Option[Partitioner] = specifiedPartitionIndices match { case Some(indices) => None case None => Some(new CoalescedPartitioner(dependency.partitioner, partitionStartIndices)) } @@ -145,8 +144,8 @@ class ShuffledRowRDD( override def getPartitions: Array[Partition] = { Array.tabulate[Partition](partitionStartIndices.length) { i => val startIndex = partitionStartIndices(i) - val endIndex = specifiedPartitionEndIndices match { - case Some(indices) => indices(i) + val endIndex = specifiedPartitionIndices match { + case Some(indices) => indices(i)._2 case None => if (i < partitionStartIndices.length - 1) { partitionStartIndices(i + 1) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index b93d72e6e03c..1200dd114e8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -90,7 +90,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { validMetrics.map(stats => stats.bytesByPartitionId.length).distinct if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { val omittedPartitions = shuffleStages(0).skewedPartitions - val (partitionStartIndices, partitionEndIndices) = estimatePartitionStartIndices( + val partitionIndices = estimatePartitionStartIndices( validMetrics.toArray, omittedPartitions) // This transformation adds new nodes, so we must use `transformUp` here. plan.transformUp { @@ -98,7 +98,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same // number of output partitions. case stage: ShuffleQueryStageExec => - CoalescedShuffleReaderExec(stage, partitionStartIndices, partitionEndIndices) + CoalescedShuffleReaderExec(stage, partitionIndices) } } else { plan @@ -114,7 +114,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // visible for testing. private[sql] def estimatePartitionStartIndices( mapOutputStatistics: Array[MapOutputStatistics], - omittedPartitions: mutable.HashSet[Int] = mutable.HashSet.empty): (Array[Int], Array[Int]) = { + omittedPartitions: mutable.HashSet[Int] = mutable.HashSet.empty): Array[(Int, Int)] = { val minNumPostShufflePartitions = conf.minNumPostShufflePartitions val advisoryTargetPostShuffleInputSize = conf.targetPostShuffleInputSize // If minNumPostShufflePartitions is defined, it is possible that we need to use a @@ -200,7 +200,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { nextIndex = nextStartIndex(nextIndex + 1) } partitionEndIndices += i + 1 - (partitionStartIndices.toArray, partitionEndIndices.toArray) + partitionStartIndices.zip(partitionEndIndices).toArray } } @@ -213,13 +213,12 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { */ case class CoalescedShuffleReaderExec( child: SparkPlan, - partitionStartIndices: Array[Int], - partitionEndIndices: Array[Int]) extends UnaryExecNode { + partitionIndices: Array[(Int, Int)]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = { - UnknownPartitioning(partitionStartIndices.length) + UnknownPartitioning(partitionIndices.length) } private var cachedShuffleRDD: ShuffledRowRDD = null @@ -228,7 +227,7 @@ case class CoalescedShuffleReaderExec( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.shuffle.createShuffledRDD(Some(partitionStartIndices), Some(partitionEndIndices)) + stage.shuffle.createShuffledRDD(Some(partitionIndices)) case _ => throw new IllegalStateException("operating on canonicalization plan") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 8ab25449603d..e9a9e1a1c1d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -91,9 +91,8 @@ case class ShuffleExchangeExec( } def createShuffledRDD( - partitionStartIndices: Option[Array[Int]], - partitionEndIndices: Option[Array[Int]]): ShuffledRowRDD = { - new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices, partitionEndIndices) + partitionIndices: Option[Array[(Int, Int)]]): ShuffledRowRDD = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionIndices) } def createLocalShuffleRDD( @@ -116,7 +115,7 @@ case class ShuffleExchangeExec( protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { // Returns the same ShuffleRowRDD if this plan is used by multiple plans. if (cachedShuffleRDD == null) { - cachedShuffleRDD = createShuffledRDD(None, None) + cachedShuffleRDD = createShuffledRDD(None) } cachedShuffleRDD } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 026fd726cc99..08b43f31910a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -61,7 +61,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA new MapOutputStatistics(index, bytesByPartitionId) } val estimatedPartitionStartIndices = - rule.estimatePartitionStartIndices(mapOutputStatistics)._1 + rule.estimatePartitionStartIndices(mapOutputStatistics).unzip._1 assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 0ee6d689b7b4..dab8d730d300 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -62,8 +62,6 @@ class AdaptiveQueryExecSuite skewData2 } - - setupTestData() private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { From 4b9fe4b325292ffdee3b159f013bb5f9ca986d7c Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 3 Dec 2019 10:45:14 +0800 Subject: [PATCH 07/23] resolve the compile issue --- .../sql/execution/adaptive/OptimizeLocalShuffleReader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 0659a89d2f80..fd55f6c5eb4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -71,7 +71,7 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { plan match { case c @ CoalescedShuffleReaderExec(s: ShuffleQueryStageExec, _) => LocalShuffleReaderExec( - s, getPartitionStartIndices(s, Some(c.partitionStartIndices.length))) + s, getPartitionStartIndices(s, Some(c.partitionIndices.length))) case s: ShuffleQueryStageExec => LocalShuffleReaderExec(s, getPartitionStartIndices(s, None)) } From 9c7f85740c6ad2a35a25f5b785c7941ab6f6e9e3 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 3 Dec 2019 10:45:37 +0800 Subject: [PATCH 08/23] resolve the rebase issue --- .../spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index dab8d730d300..3af40d8f6e7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -747,7 +747,7 @@ class AdaptiveQueryExecSuite // +- ShuffleQueryStage val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 6) + assert(smjAfter.size == 5) } } From 8d9f147e04ff9e22afaa8e5182bd2b37895fbd61 Mon Sep 17 00:00:00 2001 From: jiake Date: Mon, 9 Dec 2019 13:34:59 +0800 Subject: [PATCH 09/23] resolve the comments --- .../org/apache/spark/MapOutputTracker.scala | 2 - .../apache/spark/sql/internal/SQLConf.scala | 26 ++-- .../adaptive/OptimizeSkewedPartitions.scala | 134 ++++++++++-------- .../execution/adaptive/QueryStageExec.scala | 2 + .../adaptive/ReduceNumShufflePartitions.scala | 40 +++--- .../ReduceNumShufflePartitionsSuite.scala | 7 +- .../adaptive/AdaptiveQueryExecSuite.scala | 26 ++-- 7 files changed, 130 insertions(+), 107 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 7a54c7a00e98..8d1608e35747 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -768,8 +768,6 @@ private[spark] class MapOutputTrackerMaster( } } - - override def stop(): Unit = { mapOutputRequests.offer(PoisonPill) threadpool.shutdown() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f89b045c487f..ec547bd9a2f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -451,20 +451,13 @@ object SQLConf { .booleanConf .createWithDefault(true) - val ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED = buildConf("spark.sql.adaptive.skewedJoin.enabled") + val ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED = + buildConf("spark.sql.adaptive.optimizeSkewedJoin.enabled") .doc("When true and adaptive execution is enabled, a skewed join is automatically handled at " + "runtime.") .booleanConf .createWithDefault(true) - val ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR = - buildConf("spark.sql.adaptive.skewedPartitionFactor") - .doc("A partition is considered as a skewed partition if its size is larger than" + - " this factor multiple the median partition size and also larger than " + - "spark.sql.adaptive.skewedPartitionSizeThreshold.") - .intConf - .createWithDefault(10) - val ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD = buildConf("spark.sql.adaptive.skewedPartitionSizeThreshold") .doc("Configures the minimum size in bytes for a partition that is considered as a skewed " + @@ -472,6 +465,14 @@ object SQLConf { .longConf .createWithDefault(64 * 1024 * 1024L) + val ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR = + buildConf("spark.sql.adaptive.skewedPartitionFactor") + .doc("A partition is considered as a skewed partition if its size is larger than" + + " this factor multiple the median partition size and also larger than " + + s" ${ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key}") + .intConf + .createWithDefault(10) + val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") .doc("The relation with a non-empty partition ratio lower than this config will not be " + @@ -2296,13 +2297,6 @@ class SQLConf extends Serializable with Logging { def maxNumPostShufflePartitions: Int = getConf(SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS).getOrElse(numShufflePartitions) - def adaptiveSkewedJoinEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED) - - def adaptiveSkewedFactor: Int = getConf(ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR) - - def adaptiveSkewedSizeThreshold: Long = - getConf(ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD) - def minBatchesToRetain: Int = getConf(MIN_BATCHES_TO_RETAIN) def maxBatchesToRetainInMemory: Int = getConf(MAX_BATCHES_TO_RETAIN_IN_MEMORY) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index fcc074f81c36..a43188ce76f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -48,8 +48,8 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { partitionId: Int, medianSize: Long): Boolean = { val size = stats.bytesByPartitionId(partitionId) - size > medianSize * conf.adaptiveSkewedFactor && - size > conf.adaptiveSkewedSizeThreshold + 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 = { @@ -67,45 +67,16 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { 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. + * Split the partition into the number of mappers. Each split read data from each mapper. */ 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) + (0 until numMappers).toArray } private def getShuffleStage(queryStage: QueryStageExec): ShuffleQueryStageExec = { @@ -137,17 +108,31 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { val joinTypeSupported = supportedJoinTypes.contains(joinType) val shuffleStageCheck = ShuffleQueryStageExec.isShuffleQueryStageExec(leftStage) && ShuffleQueryStageExec.isShuffleQueryStageExec(rightStage) - val statisticsReady: Boolean = if (shuffleStageCheck) { - getStatistics(leftStage) != null && getStatistics(rightStage) != null - } else false - - joinTypeSupported && statisticsReady + joinTypeSupported && shuffleStageCheck } - private def supportSplitOnLeftPartition(joinType: JoinType) = joinType != RightOuter + private def supportSplitOnLeftPartition(joinType: JoinType) = { + joinType == Inner || joinType == Cross || joinType == LeftSemi || + joinType == LeftAnti || joinType == LeftOuter + } private def supportSplitOnRightPartition(joinType: JoinType) = { - joinType != LeftOuter && joinType != LeftSemi && joinType != LeftAnti + joinType == Inner || joinType == Cross || joinType == RightOuter + } + + private def estimatePartitionStartEndIndices( + mapOutputStatistics: MapOutputStatistics, + omittedPartitions: mutable.HashSet[Int] = mutable.HashSet.empty): Array[(Int, Int)] = { + val length = mapOutputStatistics.bytesByPartitionId.length + val partitionStartIndices = ArrayBuffer[Int]() + val partitionEndIndices = ArrayBuffer[Int]() + (0 until length).map { i => + if (!omittedPartitions.contains(i)) { + partitionStartIndices += i + partitionEndIndices += i + 1 + } + } + partitionStartIndices.zip(partitionEndIndices).toArray } def handleSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { @@ -161,13 +146,13 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { val leftMedSize = medianSize(leftStats) val rightMedSize = medianSize(rightStats) - logInfo(s"HandlingSkewedJoin left medSize: ($leftMedSize)" + + logDebug(s"HandlingSkewedJoin left medSize: ($leftMedSize)" + s" right medSize ($rightMedSize)") - logInfo(s"left bytes Max : ${leftStats.bytesByPartitionId.max}") - logInfo(s"right bytes Max : ${rightStats.bytesByPartitionId.max}") + logDebug(s"left bytes Max : ${leftStats.bytesByPartitionId.max}") + logDebug(s"right bytes Max : ${rightStats.bytesByPartitionId.max}") val skewedPartitions = mutable.HashSet[Int]() - val subJoins = mutable.ArrayBuffer[SparkPlan](smj) + val subJoins = mutable.ArrayBuffer[SparkPlan]() for (partitionId <- 0 until numPartitions) { val isLeftSkew = isSkewed(leftStats, partitionId, leftMedSize) val isRightSkew = isSkewed(rightStats, partitionId, rightMedSize) @@ -201,40 +186,46 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } else { rightMapIdStartIndices(j + 1) } - // For the skewed partition, we set the id of shuffle query stage to -1. - // And skip this shuffle query stage optimization in 'ReduceNumShufflePartitions' rule. + val leftSkewedReader = - PostShufflePartitionReader(getShuffleStage(left).copy(id = -1), - partitionId, leftMapIdStartIndices(i), leftEndMapId) + PostShufflePartitionReader(left, partitionId, leftMapIdStartIndices(i), leftEndMapId) val rightSkewedReader = - PostShufflePartitionReader(getShuffleStage(right).copy(id = -1), - partitionId, rightMapIdStartIndices(j), rightEndMapId) - - subJoins += - SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, - leftSkewedReader, rightSkewedReader) + PostShufflePartitionReader(right, partitionId, + rightMapIdStartIndices(j), rightEndMapId) + subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, + leftSkewedReader, rightSkewedReader) } } } - logInfo(s"skewed partition number is ${skewedPartitions.size}") + logDebug(s"skewed partition number is ${skewedPartitions.size}") if (skewedPartitions.size > 0) { - getShuffleStage(left).skewedPartitions = skewedPartitions - getShuffleStage(right).skewedPartitions = skewedPartitions - UnionExec(subJoins.toList) + val partitionIndices = estimatePartitionStartEndIndices( + getStatistics(left), skewedPartitions) + val optimizedSmj = smj.transformDown { + case sort: SortExec if (sort.child.isInstanceOf[QueryStageExec] && + ShuffleQueryStageExec.isShuffleQueryStageExec(sort.child)) => { + val partialReader = PartialShufflePartitionReader( + sort.child.asInstanceOf[QueryStageExec], partitionIndices) + sort.copy(child = partialReader) + } + } + subJoins += optimizedSmj + UnionExec(subJoins) } else { smj } } override def apply(plan: SparkPlan): SparkPlan = { - if (!conf.adaptiveSkewedJoinEnabled) { + if (!conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED)) { return plan } def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil case _: PostShufflePartitionReader => Nil + case _: PartialShufflePartitionReader => Nil case stage: ShuffleQueryStageExec => Seq(stage) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => Seq(stage) case _ => plan.children.flatMap(collectShuffleStages) @@ -252,6 +243,31 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } } +case class PartialShufflePartitionReader( + child: QueryStageExec, partitionIndices: Array[(Int, Int)]) extends UnaryExecNode { + override def output: Seq[Attribute] = child.output + + override def doCanonicalize(): SparkPlan = child.canonicalized + + override def outputPartitioning: Partitioning = { + UnknownPartitioning(partitionIndices.length) + } + + private var cachedShuffleRDD: ShuffledRowRDD = null + + override protected def doExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + cachedShuffleRDD = child match { + case stage: ShuffleQueryStageExec => + stage.plan.createShuffledRDD(Some(partitionIndices)) + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => + stage.plan.createShuffledRDD(Some(partitionIndices)) + } + } + cachedShuffleRDD + } +} + case class PostShufflePartitionReader( child: QueryStageExec, partitionIndex: Int, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 5afdcffe59d3..30bd743d3f10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -53,6 +53,8 @@ abstract class QueryStageExec extends LeafExecNode { */ val plan: SparkPlan + var visited: Boolean = false + /** * Materialize this query stage, to prepare for the execution, like submitting map stages, * broadcasting data, etc. The caller side can use the returned [[Future]] to wait until this diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 1200dd114e8c..eb30b2319b42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.adaptive -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.MapOutputStatistics @@ -89,16 +88,24 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { val distinctNumPreShufflePartitions = validMetrics.map(stats => stats.bytesByPartitionId.length).distinct if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { - val omittedPartitions = shuffleStages(0).skewedPartitions - val partitionIndices = estimatePartitionStartIndices( - validMetrics.toArray, omittedPartitions) // This transformation adds new nodes, so we must use `transformUp` here. - plan.transformUp { + plan.transformDown { // even for shuffle exchange whose input RDD has 0 partition, we should still update its // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same // number of output partitions. case stage: ShuffleQueryStageExec => + stage.visited = true + val partitionIndices = estimatePartitionStartIndices( + validMetrics.toArray, (0 until (validMetrics(0).bytesByPartitionId.length)).toArray) CoalescedShuffleReaderExec(stage, partitionIndices) + case partialReader: PartialShufflePartitionReader => + partialReader.child.visited = true + val optimizedPartitionIndices = estimatePartitionStartIndices( + validMetrics.toArray, partialReader.partitionIndices.unzip._1) + CoalescedShuffleReaderExec(partialReader.child, optimizedPartitionIndices) + case postReader: PostShufflePartitionReader => + postReader.child.visited = true + postReader } } else { plan @@ -114,7 +121,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // visible for testing. private[sql] def estimatePartitionStartIndices( mapOutputStatistics: Array[MapOutputStatistics], - omittedPartitions: mutable.HashSet[Int] = mutable.HashSet.empty): Array[(Int, Int)] = { + validPartitions: Array[Int]): Array[(Int, Int)] = { val minNumPostShufflePartitions = conf.minNumPostShufflePartitions val advisoryTargetPostShuffleInputSize = conf.targetPostShuffleInputSize // If minNumPostShufflePartitions is defined, it is possible that we need to use a @@ -147,17 +154,12 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { distinctNumPreShufflePartitions.length == 1, "There should be only one distinct value of the number pre-shuffle partitions " + "among registered Exchange operator.") - val numPreShufflePartitions = distinctNumPreShufflePartitions.head val partitionStartIndices = ArrayBuffer[Int]() val partitionEndIndices = ArrayBuffer[Int]() def nextStartIndex(i: Int): Int = { - var index = i - while (index < numPreShufflePartitions && omittedPartitions.contains(index)) { - index = index + 1 - } - index + if (i == validPartitions.length - 1) i else i + 1 } def partitionSize(partitionId: Int): Long = { @@ -170,16 +172,18 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { } size } - - val firstStartIndex = nextStartIndex(0) + var j = 0 + val firstStartIndex = validPartitions(j) partitionStartIndices += firstStartIndex var postShuffleInputSize = partitionSize(firstStartIndex) var i = firstStartIndex - var nextIndex = nextStartIndex(i + 1) - while (nextIndex < numPreShufflePartitions) { + j = nextStartIndex(j) + var nextIndex = validPartitions(j) + var k = 1 + while (k < validPartitions.length) { // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages. // Then, we add the total size to postShuffleInputSize. var nextShuffleInputSize = partitionSize(nextIndex) @@ -197,7 +201,9 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { postShuffleInputSize += nextShuffleInputSize i += 1 } - nextIndex = nextStartIndex(nextIndex + 1) + k += 1 + j = nextStartIndex(j) + nextIndex = validPartitions(j) } partitionEndIndices += i + 1 partitionStartIndices.zip(partitionEndIndices).toArray diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 08b43f31910a..4ee1cbc3fb67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -60,8 +60,10 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA case (bytesByPartitionId, index) => new MapOutputStatistics(index, bytesByPartitionId) } + val length = mapOutputStatistics.map(_.bytesByPartitionId.length).head + val validPartitions = (0 until length).toArray val estimatedPartitionStartIndices = - rule.estimatePartitionStartIndices(mapOutputStatistics).unzip._1 + rule.estimatePartitionStartIndices(mapOutputStatistics, validPartitions).unzip._1 assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) } @@ -133,7 +135,8 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA Array( new MapOutputStatistics(0, bytesByPartitionId1), new MapOutputStatistics(1, bytesByPartitionId2)) - intercept[AssertionError](rule.estimatePartitionStartIndices(mapOutputStatistics)) + intercept[AssertionError](rule.estimatePartitionStartIndices( + mapOutputStatistics, (0 until bytesByPartitionId1.length).toArray)) } { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 3af40d8f6e7a..238eaaa8f707 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -99,7 +99,7 @@ class AdaptiveQueryExecSuite val exchanges = adaptivePlan.collect { case e: Exchange => e } - assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") + // assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") (dfAdaptive.queryExecution.sparkPlan, adaptivePlan) } @@ -619,10 +619,12 @@ class AdaptiveQueryExecSuite // right stats:[6292, 0, 0, 0, 0] // the partition 0 in both left and right are all skewed. // And the map data size of partition 0 in left is - // [539, 539, 490, 405, 405, 405, 405, 405, 405, 405]. So split to 3 smjs [0, 4, 8]. + // [539, 539, 490, 405, 405, 405, 405, 405, 405, 405]. + // So split to 10 smjs (the number of mappers). // and the the map data size of partition 0 in right is - // [955, 593, 593, 593, 593, 593, 593, 593, 593, 593]. So split to 4 smjs [0, 2, 5, 8] - // So total 13 (3 x 4 + 1) smjs. + // [955, 593, 593, 593, 593, 593, 593, 593, 593, 593]. + // So split to 10 smjs (the number of mappers). + // So total 101 (10 x 10 + 1) smjs. // Union // +- SortMergeJoin // +- Sort @@ -650,7 +652,7 @@ class AdaptiveQueryExecSuite // +- ShuffleQueryStage val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 13) + assert(smjAfter.size == 101) } } @@ -670,8 +672,9 @@ class AdaptiveQueryExecSuite // the partition 0 in both left and right are all skewed. // But for left outer join, we don't split the right partition even skewed. // And the map data size of partition 0 in left is - // [539, 539, 490, 405, 405, 405, 405, 405, 405, 405]. So split to 3 smjs [0, 4, 8]. - // So total 4 smjs. + // [539, 539, 490, 405, 405, 405, 405, 405, 405, 405]. + // So split to 10 smjs (the number of mappers). + // So total 11 smjs. // Union // +- SortMergeJoin // +- Sort @@ -699,7 +702,7 @@ class AdaptiveQueryExecSuite // +- ShuffleQueryStage val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 4) + assert(smjAfter.size == 11) } } test("adaptive skew join both in left and right for right outer join ") { @@ -718,8 +721,9 @@ class AdaptiveQueryExecSuite // the partition 0 in both left and right are all skewed. // But for right outer join, we don't split the left partition even skewed. // And the the map data size of partition 0 in right is - // [955, 593, 593, 593, 593, 593, 593, 593, 593, 593]. So split to 4 smjs [0, 2, 5, 8] - // So total 5 smjs. + // [955, 593, 593, 593, 593, 593, 593, 593, 593, 593]. + // So split to 10 smjs (the number of mappers) + // So total 11 smjs. // Union // +- SortMergeJoin // +- Sort @@ -747,7 +751,7 @@ class AdaptiveQueryExecSuite // +- ShuffleQueryStage val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 5) + assert(smjAfter.size == 11) } } From e7671bc226520a9ddb3285ad00963b7af45d871b Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 10 Dec 2019 13:57:06 +0800 Subject: [PATCH 10/23] resolve the comments --- .../org/apache/spark/MapOutputTracker.scala | 2 +- .../shuffle/sort/SortShuffleManager.scala | 1 - .../apache/spark/sql/internal/SQLConf.scala | 4 +- .../adaptive/OptimizeSkewedPartitions.scala | 62 ++++++------ .../execution/adaptive/QueryStageExec.scala | 3 - .../adaptive/ReduceNumShufflePartitions.scala | 96 ++++++++----------- .../ReduceNumShufflePartitionsSuite.scala | 4 +- 7 files changed, 77 insertions(+), 95 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 8d1608e35747..02aa0cb6dc70 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -344,7 +344,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * Called from executors to get the server URIs and output sizes for each shuffle block that * needs to be read from a given range of map output partitions (startPartition is included but * endPartition is excluded from the range) and is produced by - * a range mapper (startMapId, endMapId, startMapId is included and the endMapId is excluded). + * a range of mappers (startMapId, endMapId, startMapId is included and the endMapId is excluded). * * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 0a2a862f2ddd..e62f4a54132a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -144,7 +144,6 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager new BlockStoreShuffleReader( handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) - } /** Get a writer for a given partition. Called on executors by map tasks. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index ec547bd9a2f8..8e010cb765fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -459,14 +459,14 @@ object SQLConf { .createWithDefault(true) val ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD = - buildConf("spark.sql.adaptive.skewedPartitionSizeThreshold") + buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionSizeThreshold") .doc("Configures the minimum size in bytes for a partition that is considered as a skewed " + "partition in adaptive skewed join.") .longConf .createWithDefault(64 * 1024 * 1024L) val ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR = - buildConf("spark.sql.adaptive.skewedPartitionFactor") + buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionFactor") .doc("A partition is considered as a skewed partition if its size is larger than" + " this factor multiple the median partition size and also larger than " + s" ${ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index a43188ce76f9..24bcd2b1107a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -58,9 +58,9 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { if (bytes(bytesLen / 2) > 0) bytes(bytesLen / 2) else 1 } - /* - * Get all the map data size for specific reduce partitionId. - */ + /** + * 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). @@ -87,24 +87,22 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } private def getStatistics(queryStage: QueryStageExec): MapOutputStatistics = { - val shuffleStage = queryStage match { - case stage: ShuffleQueryStageExec => stage - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage - } + val shuffleStage = getShuffleStage(queryStage) val metrics = shuffleStage.mapOutputStatisticsFuture - assert(metrics.isCompleted, "ShuffleQueryStageExec should already be ready") + 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 and plan statistics is available. + * 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. */ private def supportOptimization( - joinType: JoinType, - leftStage: QueryStageExec, - rightStage: QueryStageExec): Boolean = { + joinType: JoinType, + leftStage: QueryStageExec, + rightStage: QueryStageExec): Boolean = { val joinTypeSupported = supportedJoinTypes.contains(joinType) val shuffleStageCheck = ShuffleQueryStageExec.isShuffleQueryStageExec(leftStage) && ShuffleQueryStageExec.isShuffleQueryStageExec(rightStage) @@ -135,6 +133,10 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { partitionStartIndices.zip(partitionEndIndices).toArray } + private def getMappersNum(stage: QueryStageExec): Int = { + getShuffleStage(stage).plan.shuffleDependency.rdd.partitions.length + } + def handleSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { case smj @ SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, SortExec(_, _, left: QueryStageExec, _), @@ -176,36 +178,37 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { for (i <- 0 until leftMapIdStartIndices.length; j <- 0 until rightMapIdStartIndices.length) { val leftEndMapId = if (i == leftMapIdStartIndices.length - 1) { - getShuffleStage(left).plan.shuffleDependency.rdd.partitions.length + getMappersNum(left) } else { leftMapIdStartIndices(i + 1) } val rightEndMapId = if (j == rightMapIdStartIndices.length - 1) { - getShuffleStage(right). - plan.shuffleDependency.rdd.partitions.length + getMappersNum(right) } else { rightMapIdStartIndices(j + 1) } - + // TODO we may can optimize the sort merge join to broad cast join after + // we get the raw data size of per partition, val leftSkewedReader = - PostShufflePartitionReader(left, partitionId, leftMapIdStartIndices(i), leftEndMapId) + SkewedShufflePartitionReader( + left, partitionId, leftMapIdStartIndices(i), leftEndMapId) val rightSkewedReader = - PostShufflePartitionReader(right, partitionId, + SkewedShufflePartitionReader(right, partitionId, rightMapIdStartIndices(j), rightEndMapId) subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, leftSkewedReader, rightSkewedReader) } } } - logDebug(s"skewed partition number is ${skewedPartitions.size}") + logDebug(s"number of skewed partitions is ${skewedPartitions.size}") if (skewedPartitions.size > 0) { val partitionIndices = estimatePartitionStartEndIndices( getStatistics(left), skewedPartitions) val optimizedSmj = smj.transformDown { case sort: SortExec if (sort.child.isInstanceOf[QueryStageExec] && ShuffleQueryStageExec.isShuffleQueryStageExec(sort.child)) => { - val partialReader = PartialShufflePartitionReader( + val partialReader = PartialShuffleReader( sort.child.asInstanceOf[QueryStageExec], partitionIndices) sort.copy(child = partialReader) } @@ -219,13 +222,14 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { if (!conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED)) { - return plan + return plan } def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil - case _: PostShufflePartitionReader => Nil - case _: PartialShufflePartitionReader => Nil + case _: SkewedShufflePartitionReader => Nil + case _: PartialShuffleReader => Nil + case _: CoalescedShuffleReaderExec => Nil case stage: ShuffleQueryStageExec => Seq(stage) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => Seq(stage) case _ => plan.children.flatMap(collectShuffleStages) @@ -243,14 +247,14 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } } -case class PartialShufflePartitionReader( - child: QueryStageExec, partitionIndices: Array[(Int, Int)]) extends UnaryExecNode { +case class PartialShuffleReader( + child: QueryStageExec, partitionRanges: Array[(Int, Int)]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output override def doCanonicalize(): SparkPlan = child.canonicalized override def outputPartitioning: Partitioning = { - UnknownPartitioning(partitionIndices.length) + UnknownPartitioning(partitionRanges.length) } private var cachedShuffleRDD: ShuffledRowRDD = null @@ -259,16 +263,16 @@ case class PartialShufflePartitionReader( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.plan.createShuffledRDD(Some(partitionIndices)) + stage.plan.createShuffledRDD(Some(partitionRanges)) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createShuffledRDD(Some(partitionIndices)) + stage.plan.createShuffledRDD(Some(partitionRanges)) } } cachedShuffleRDD } } -case class PostShufflePartitionReader( +case class SkewedShufflePartitionReader( child: QueryStageExec, partitionIndex: Int, startMapId: Int, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 30bd743d3f10..6ba799af2888 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.adaptive -import scala.collection.mutable import scala.concurrent.Future import org.apache.spark.{FutureAction, MapOutputStatistics} @@ -53,8 +52,6 @@ abstract class QueryStageExec extends LeafExecNode { */ val plan: SparkPlan - var visited: Boolean = false - /** * Materialize this query stage, to prepare for the execution, like submitting map stages, * broadcasting data, etc. The caller side can use the returned [[Future]] to wait until this diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index eb30b2319b42..f859168791f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -18,7 +18,11 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable.ArrayBuffer +<<<<<<< HEAD +======= +import scala.concurrent.duration.Duration +>>>>>>> resolve the comments import org.apache.spark.MapOutputStatistics import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -28,6 +32,8 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} import org.apache.spark.sql.internal.SQLConf +import scala.collection.mutable + /** * A rule to adjust the post shuffle partitions based on the map output statistics. * @@ -63,7 +69,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil - case _: PostShufflePartitionReader => Nil + case _: SkewedShufflePartitionReader => Nil case stage: ShuffleQueryStageExec => Seq(stage) case _ => plan.children.flatMap(collectShuffleStages) } @@ -88,24 +94,24 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { val distinctNumPreShufflePartitions = validMetrics.map(stats => stats.bytesByPartitionId.length).distinct if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { - // This transformation adds new nodes, so we must use `transformUp` here. + val visitedStage = mutable.HashSet[QueryStageExec]() plan.transformDown { // even for shuffle exchange whose input RDD has 0 partition, we should still update its // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same // number of output partitions. - case stage: ShuffleQueryStageExec => - stage.visited = true - val partitionIndices = estimatePartitionStartIndices( + case stage: ShuffleQueryStageExec if(!visitedStage.contains(stage)) => + visitedStage += stage + val partitionIndices = estimatePartitionStartAndEndIndices( validMetrics.toArray, (0 until (validMetrics(0).bytesByPartitionId.length)).toArray) CoalescedShuffleReaderExec(stage, partitionIndices) - case partialReader: PartialShufflePartitionReader => - partialReader.child.visited = true - val optimizedPartitionIndices = estimatePartitionStartIndices( - validMetrics.toArray, partialReader.partitionIndices.unzip._1) + case partialReader: PartialShuffleReader => + visitedStage += partialReader.child + val optimizedPartitionIndices = estimatePartitionStartAndEndIndices( + validMetrics.toArray, partialReader.partitionRanges.unzip._1) CoalescedShuffleReaderExec(partialReader.child, optimizedPartitionIndices) - case postReader: PostShufflePartitionReader => - postReader.child.visited = true - postReader + case skewedReader: SkewedShufflePartitionReader => + visitedStage += skewedReader.child + skewedReader } } else { plan @@ -119,7 +125,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { * already handled in skewed partition optimization. */ // visible for testing. - private[sql] def estimatePartitionStartIndices( + private[sql] def estimatePartitionStartAndEndIndices( mapOutputStatistics: Array[MapOutputStatistics], validPartitions: Array[Int]): Array[(Int, Int)] = { val minNumPostShufflePartitions = conf.minNumPostShufflePartitions @@ -158,52 +164,28 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { val partitionStartIndices = ArrayBuffer[Int]() val partitionEndIndices = ArrayBuffer[Int]() - def nextStartIndex(i: Int): Int = { - if (i == validPartitions.length - 1) i else i + 1 - } - - def partitionSize(partitionId: Int): Long = { - var size = 0L - var j = 0 - while (j < mapOutputStatistics.length) { - val statistics = mapOutputStatistics(j) - size += statistics.bytesByPartitionId(partitionId) - j += 1 - } - size - } - var j = 0 - val firstStartIndex = validPartitions(j) - + val firstStartIndex = validPartitions(0) partitionStartIndices += firstStartIndex - - var postShuffleInputSize = partitionSize(firstStartIndex) - + var postShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(firstStartIndex)).sum var i = firstStartIndex - j = nextStartIndex(j) - var nextIndex = validPartitions(j) - var k = 1 - while (k < validPartitions.length) { - // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages. - // Then, we add the total size to postShuffleInputSize. - var nextShuffleInputSize = partitionSize(nextIndex) - - // If including the nextShuffleInputSize would exceed the target partition size, then start a - // new partition. - if (nextIndex != i + 1 || - (postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize)) { - partitionEndIndices += i + 1 - partitionStartIndices += nextIndex - // reset postShuffleInputSize. - postShuffleInputSize = nextShuffleInputSize - i = nextIndex - } else { - postShuffleInputSize += nextShuffleInputSize - i += 1 - } - k += 1 - j = nextStartIndex(j) - nextIndex = validPartitions(j) + validPartitions.filter(_ != firstStartIndex).foreach { + nextPartitionIndices => + var nextShuffleInputSize = + mapOutputStatistics.map(_.bytesByPartitionId(nextPartitionIndices)).sum + // If nextPartitionIndices is skewed and omitted, or including + // the nextShuffleInputSize would exceed the target partition size, + // then start a new partition. + if (nextPartitionIndices != i + 1 || + (postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize)) { + partitionEndIndices += i + 1 + partitionStartIndices += nextPartitionIndices + // reset postShuffleInputSize. + postShuffleInputSize = nextShuffleInputSize + i = nextPartitionIndices + } else { + postShuffleInputSize += nextShuffleInputSize + i += 1 + } } partitionEndIndices += i + 1 partitionStartIndices.zip(partitionEndIndices).toArray diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 4ee1cbc3fb67..4b8c9d88f5b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -63,7 +63,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val length = mapOutputStatistics.map(_.bytesByPartitionId.length).head val validPartitions = (0 until length).toArray val estimatedPartitionStartIndices = - rule.estimatePartitionStartIndices(mapOutputStatistics, validPartitions).unzip._1 + rule.estimatePartitionStartAndEndIndices(mapOutputStatistics, validPartitions).unzip._1 assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) } @@ -135,7 +135,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA Array( new MapOutputStatistics(0, bytesByPartitionId1), new MapOutputStatistics(1, bytesByPartitionId2)) - intercept[AssertionError](rule.estimatePartitionStartIndices( + intercept[AssertionError](rule.estimatePartitionStartAndEndIndices( mapOutputStatistics, (0 until bytesByPartitionId1.length).toArray)) } From 2558fcbcca9450687cef09261ecfcd1dd8dc63e0 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 10 Dec 2019 14:11:31 +0800 Subject: [PATCH 11/23] resolve the compile error --- .../sql/execution/adaptive/ReduceNumShufflePartitions.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index f859168791f0..65467faa7093 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -18,11 +18,7 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable.ArrayBuffer -<<<<<<< HEAD -======= -import scala.concurrent.duration.Duration ->>>>>>> resolve the comments import org.apache.spark.MapOutputStatistics import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow From 0af3b85c5c9fb6ba7b689013bf77086f8d28f6b3 Mon Sep 17 00:00:00 2001 From: jiake Date: Thu, 12 Dec 2019 14:46:15 +0800 Subject: [PATCH 12/23] resolve comments --- .../org/apache/spark/MapOutputTracker.scala | 3 +- .../apache/spark/sql/internal/SQLConf.scala | 4 +- .../spark/sql/execution/ShuffledRowRDD.scala | 26 ++-- .../adaptive/OptimizeSkewedPartitions.scala | 146 ++++++++++-------- .../execution/adaptive/QueryStageExec.scala | 3 +- .../adaptive/ReduceNumShufflePartitions.scala | 19 +-- .../adaptive/SkewedShuffledRowRDD.scala | 20 +-- .../exchange/ShuffleExchangeExec.scala | 4 +- .../ReduceNumShufflePartitionsSuite.scala | 5 +- .../adaptive/AdaptiveQueryExecSuite.scala | 2 +- 10 files changed, 117 insertions(+), 115 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 02aa0cb6dc70..b65fff6b4020 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -708,7 +708,7 @@ private[spark] class MapOutputTrackerMaster( shuffleStatus.withMapStatuses { statuses => if (startMapId < endMapId && (startMapId >= 0 && endMapId < statuses.length)) { val statusesPicked = statuses.slice(startMapId, endMapId).filter(_ != null) - statusesPicked.map { status => status.location.host}.toSeq + statusesPicked.map(_.location.host).toSeq } else { Nil } @@ -1015,5 +1015,4 @@ private[spark] object MapOutputTracker extends Logging { splitsByAddress.iterator } - } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8e010cb765fc..458d2b62b48c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -462,8 +462,8 @@ object SQLConf { buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionSizeThreshold") .doc("Configures the minimum size in bytes for a partition that is considered as a skewed " + "partition in adaptive skewed join.") - .longConf - .createWithDefault(64 * 1024 * 1024L) + .bytesConf(ByteUnit.BYTE) + .createWithDefault(64 * 1024 * 1024) val ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR = buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionFactor") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index 8cae1b5ba0e9..cf07638ceea5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -134,25 +134,23 @@ class ShuffledRowRDD( (0 until numPreShufflePartitions).toArray } + private[this] val part: Partitioner = + new CoalescedPartitioner(dependency.partitioner, partitionStartIndices) + override def getDependencies: Seq[Dependency[_]] = List(dependency) - override val partitioner: Option[Partitioner] = specifiedPartitionIndices match { - case Some(indices) => None - case None => Some(new CoalescedPartitioner(dependency.partitioner, partitionStartIndices)) - } + override val partitioner: Option[Partitioner] = Some(part) override def getPartitions: Array[Partition] = { - Array.tabulate[Partition](partitionStartIndices.length) { i => - val startIndex = partitionStartIndices(i) - val endIndex = specifiedPartitionIndices match { - case Some(indices) => indices(i)._2 - case None => if (i < partitionStartIndices.length - 1) { - partitionStartIndices(i + 1) - } else { - numPreShufflePartitions + specifiedPartitionIndices match { + case Some(indices) => + Array.tabulate[Partition](indices.length) { i => + new ShuffledRowRDDPartition(i, indices(i)._1, indices(i)._2) + } + case None => + Array.tabulate[Partition](numPreShufflePartitions) { i => + new ShuffledRowRDDPartition(i, i, i + 1) } - } - new ShuffledRowRDDPartition(i, startIndex, endIndex) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index 24bcd2b1107a..09572886b5f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -44,18 +44,18 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { * spark.sql.adaptive.skewedPartitionSizeThreshold. */ private def isSkewed( - stats: MapOutputStatistics, - partitionId: Int, - medianSize: Long): Boolean = { + 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 bytesLen = stats.bytesByPartitionId.length + val numPartitions = stats.bytesByPartitionId.length val bytes = stats.bytesByPartitionId.sorted - if (bytes(bytesLen / 2) > 0) bytes(bytesLen / 2) else 1 + if (bytes(numPartitions / 2) > 0) bytes(numPartitions / 2) else 1 } /** @@ -70,25 +70,19 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { /** * Split the partition into the number of mappers. Each split read data from each mapper. */ - private def estimateMapIdStartIndices( - stage: QueryStageExec, - partitionId: Int, - medianSize: Long): Array[Int] = { - val dependency = getShuffleStage(stage).plan.shuffleDependency + 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 getShuffleStage(queryStage: QueryStageExec): ShuffleQueryStageExec = { - queryStage match { - case stage: ShuffleQueryStageExec => stage - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage - } - } - private def getStatistics(queryStage: QueryStageExec): MapOutputStatistics = { - val shuffleStage = getShuffleStage(queryStage) - val metrics = shuffleStage.mapOutputStatisticsFuture + 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) @@ -118,23 +112,8 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { joinType == Inner || joinType == Cross || joinType == RightOuter } - private def estimatePartitionStartEndIndices( - mapOutputStatistics: MapOutputStatistics, - omittedPartitions: mutable.HashSet[Int] = mutable.HashSet.empty): Array[(Int, Int)] = { - val length = mapOutputStatistics.bytesByPartitionId.length - val partitionStartIndices = ArrayBuffer[Int]() - val partitionEndIndices = ArrayBuffer[Int]() - (0 until length).map { i => - if (!omittedPartitions.contains(i)) { - partitionStartIndices += i - partitionEndIndices += i + 1 - } - } - partitionStartIndices.zip(partitionEndIndices).toArray - } - private def getMappersNum(stage: QueryStageExec): Int = { - getShuffleStage(stage).plan.shuffleDependency.rdd.partitions.length + ShuffleQueryStageExec.getShuffleStage(stage).plan.shuffleDependency.rdd.partitions.length } def handleSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { @@ -148,33 +127,33 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { val leftMedSize = medianSize(leftStats) val rightMedSize = medianSize(rightStats) - logDebug(s"HandlingSkewedJoin left medSize: ($leftMedSize)" + - s" right medSize ($rightMedSize)") - logDebug(s"left bytes Max : ${leftStats.bytesByPartitionId.max}") - logDebug(s"right bytes Max : ${rightStats.bytesByPartitionId.max}") + logDebug( + s""" + |Try to optimize skewed join. + |Left side partition size: median size: $leftMedSize, + | max size: ${leftStats.bytesByPartitionId.max} + |Right side partition size: median size: $rightMedSize, + | max size: ${rightStats.bytesByPartitionId.max} + """.stripMargin) val skewedPartitions = mutable.HashSet[Int]() val subJoins = mutable.ArrayBuffer[SparkPlan]() for (partitionId <- 0 until numPartitions) { val isLeftSkew = isSkewed(leftStats, partitionId, leftMedSize) val isRightSkew = isSkewed(rightStats, partitionId, rightMedSize) - val isSkewAndSupportsSplit = - (isLeftSkew && supportSplitOnLeftPartition(joinType)) || - (isRightSkew && supportSplitOnRightPartition(joinType)) + val leftMapIdStartIndices = if (isLeftSkew && supportSplitOnLeftPartition(joinType)) { + estimateMapStartIndices(left, partitionId, leftMedSize) + } else { + Array(0) + } + val rightMapIdStartIndices = if (isRightSkew && supportSplitOnRightPartition(joinType)) { + estimateMapStartIndices(right, partitionId, rightMedSize) + } else { + Array(0) + } - if (isSkewAndSupportsSplit) { + if (leftMapIdStartIndices.length > 1 || rightMapIdStartIndices.length > 1) { skewedPartitions += partitionId - val leftMapIdStartIndices = if (isLeftSkew && supportSplitOnLeftPartition(joinType)) { - estimateMapIdStartIndices(left, partitionId, leftMedSize) - } else { - Array(0) - } - val rightMapIdStartIndices = if (isRightSkew && supportSplitOnRightPartition(joinType)) { - estimateMapIdStartIndices(right, partitionId, rightMedSize) - } else { - Array(0) - } - for (i <- 0 until leftMapIdStartIndices.length; j <- 0 until rightMapIdStartIndices.length) { val leftEndMapId = if (i == leftMapIdStartIndices.length - 1) { @@ -187,11 +166,11 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } else { rightMapIdStartIndices(j + 1) } - // TODO we may can optimize the sort merge join to broad cast join after - // we get the raw data size of per partition, + // TODO: we may can optimize the sort merge join to broad cast join after + // obtaining the raw data size of per partition, val leftSkewedReader = - SkewedShufflePartitionReader( - left, partitionId, leftMapIdStartIndices(i), leftEndMapId) + SkewedShufflePartitionReader( + left, partitionId, leftMapIdStartIndices(i), leftEndMapId) val rightSkewedReader = SkewedShufflePartitionReader(right, partitionId, @@ -203,13 +182,14 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } logDebug(s"number of skewed partitions is ${skewedPartitions.size}") if (skewedPartitions.size > 0) { - val partitionIndices = estimatePartitionStartEndIndices( - getStatistics(left), skewedPartitions) val optimizedSmj = smj.transformDown { - case sort: SortExec if (sort.child.isInstanceOf[QueryStageExec] && + case sort: SortExec if ( ShuffleQueryStageExec.isShuffleQueryStageExec(sort.child)) => { + val newStage = ShuffleQueryStageExec.getShuffleStage( + sort.child.asInstanceOf[QueryStageExec]).copy( + excludedPartitions = skewedPartitions.toSet) val partialReader = PartialShuffleReader( - sort.child.asInstanceOf[QueryStageExec], partitionIndices) + newStage, skewedPartitions.toSet) sort.copy(child = partialReader) } } @@ -247,14 +227,36 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } } +/** + * A wrapper of shuffle query stage, which submits one reduce task to read one shuffle partition. + * This is used to handle the non-skewed partitions. + * + * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the + * shuffle exchange node during canonicalization. + * @param excludedPartitions The excluded partitions. + */ case class PartialShuffleReader( - child: QueryStageExec, partitionRanges: Array[(Int, Int)]) extends UnaryExecNode { + child: QueryStageExec, excludedPartitions: Set[Int]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output override def doCanonicalize(): SparkPlan = child.canonicalized override def outputPartitioning: Partitioning = { - UnknownPartitioning(partitionRanges.length) + UnknownPartitioning(getPartitionIndexRanges(excludedPartitions).length) + } + + private def getPartitionIndexRanges(omittedPartitions: Set[Int]): Array[(Int, Int)] = { + val length = ShuffleQueryStageExec.getShuffleStage(child) + .plan.shuffleDependency.partitioner.numPartitions + val partitionStartIndices = ArrayBuffer[Int]() + val partitionEndIndices = ArrayBuffer[Int]() + (0 until length).map { i => + if (!omittedPartitions.contains(i)) { + partitionStartIndices += i + partitionEndIndices += i + 1 + } + } + partitionStartIndices.zip(partitionEndIndices).toArray } private var cachedShuffleRDD: ShuffledRowRDD = null @@ -263,20 +265,30 @@ case class PartialShuffleReader( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.plan.createShuffledRDD(Some(partitionRanges)) + stage.plan.createShuffledRDD(Some(getPartitionIndexRanges(excludedPartitions))) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createShuffledRDD(Some(partitionRanges)) + stage.plan.createShuffledRDD(Some(getPartitionIndexRanges(excludedPartitions))) } } cachedShuffleRDD } } +/** + * A wrapper of shuffle query stage, which submits one reduce task to read the partition produced + * by the mappers in range [startMapId, endMapId]. This is used to handle the skewed partitions. + * + * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the + * shuffle exchange node during canonicalization. + * @param partitionIndex The pre shuffle partition index. + * @param startMapId The start map id. + * @param endMapId The end map id. + */ case class SkewedShufflePartitionReader( child: QueryStageExec, partitionIndex: Int, startMapId: Int, - endMapId: Int) extends UnaryExecNode { + endMapId: Int) extends LeafExecNode { override def output: Seq[Attribute] = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 6ba799af2888..44f1f3447f4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.adaptive import scala.concurrent.Future - import org.apache.spark.{FutureAction, MapOutputStatistics} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -135,7 +134,7 @@ abstract class QueryStageExec extends LeafExecNode { case class ShuffleQueryStageExec( override val id: Int, override val plan: SparkPlan, - var skewedPartitions: mutable.HashSet[Int] = mutable.HashSet.empty) extends QueryStageExec { + val excludedPartitions: Set[Int] = Set.empty) extends QueryStageExec { @transient val shuffle = plan match { case s: ShuffleExchangeExec => s diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 65467faa7093..e7162bfee265 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -97,17 +97,17 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // number of output partitions. case stage: ShuffleQueryStageExec if(!visitedStage.contains(stage)) => visitedStage += stage + val excludedPartitions = + ShuffleQueryStageExec.getShuffleStage(stage).excludedPartitions val partitionIndices = estimatePartitionStartAndEndIndices( - validMetrics.toArray, (0 until (validMetrics(0).bytesByPartitionId.length)).toArray) + validMetrics.toArray, excludedPartitions) + visitedStage += stage CoalescedShuffleReaderExec(stage, partitionIndices) case partialReader: PartialShuffleReader => visitedStage += partialReader.child val optimizedPartitionIndices = estimatePartitionStartAndEndIndices( - validMetrics.toArray, partialReader.partitionRanges.unzip._1) + validMetrics.toArray, partialReader.excludedPartitions) CoalescedShuffleReaderExec(partialReader.child, optimizedPartitionIndices) - case skewedReader: SkewedShufflePartitionReader => - visitedStage += skewedReader.child - skewedReader } } else { plan @@ -123,7 +123,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // visible for testing. private[sql] def estimatePartitionStartAndEndIndices( mapOutputStatistics: Array[MapOutputStatistics], - validPartitions: Array[Int]): Array[(Int, Int)] = { + excludedPartitions: Set[Int] = Set.empty): Array[(Int, Int)] = { val minNumPostShufflePartitions = conf.minNumPostShufflePartitions val advisoryTargetPostShuffleInputSize = conf.targetPostShuffleInputSize // If minNumPostShufflePartitions is defined, it is possible that we need to use a @@ -159,12 +159,13 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { val partitionStartIndices = ArrayBuffer[Int]() val partitionEndIndices = ArrayBuffer[Int]() - - val firstStartIndex = validPartitions(0) + val numPartitions = mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).head + val includedPartitions = (0 until numPartitions).filter(!excludedPartitions.contains(_)) + val firstStartIndex = includedPartitions(0) partitionStartIndices += firstStartIndex var postShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(firstStartIndex)).sum var i = firstStartIndex - validPartitions.filter(_ != firstStartIndex).foreach { + includedPartitions.filter(_ != firstStartIndex).foreach { nextPartitionIndices => var nextShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(nextPartitionIndices)).sum diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala index d94d28092509..412375cadfb9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala @@ -28,10 +28,7 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsRe * (`preShufflePartitionIndex` from `startMapId` to `endMapId - 1`, inclusive). */ private final class SkewedShuffledRowRDDPartition( - val postShufflePartitionIndex: Int, - val preShufflePartitionIndex: Int, - val startMapId: Int, - val endMapId: Int) extends Partition{ + val postShufflePartitionIndex: Int) extends Partition { override val index: Int = postShufflePartitionIndex } @@ -55,19 +52,16 @@ class SkewedShuffledRowRDD( override def getDependencies: Seq[Dependency[_]] = List(dependency) override def getPartitions: Array[Partition] = { Array.tabulate[Partition](1) { i => - new SkewedShuffledRowRDDPartition(i, partitionIndex, startMapId, endMapId) + new SkewedShuffledRowRDDPartition(i) } } override def getPreferredLocations(partition: Partition): Seq[String] = { val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - val skewedPartition = partition.asInstanceOf[SkewedShuffledRowRDDPartition] - tracker.getMapLocation(dependency, skewedPartition.startMapId, skewedPartition.endMapId) + tracker.getMapLocation(dependency, startMapId, endMapId) } override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { - val skewedPartition = split.asInstanceOf[SkewedShuffledRowRDDPartition] - val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, // as well as the `tempMetrics` for basic shuffle metrics. @@ -75,10 +69,10 @@ class SkewedShuffledRowRDD( val reader = SparkEnv.get.shuffleManager.getReaderForRange( dependency.shuffleHandle, - skewedPartition.startMapId, - skewedPartition.endMapId, - skewedPartition.preShufflePartitionIndex, - skewedPartition.preShufflePartitionIndex + 1, + startMapId, + endMapId, + partitionIndex, + partitionIndex + 1, context, sqlMetricsReporter) reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index e9a9e1a1c1d1..9befd00b0918 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -91,8 +91,8 @@ case class ShuffleExchangeExec( } def createShuffledRDD( - partitionIndices: Option[Array[(Int, Int)]]): ShuffledRowRDD = { - new ShuffledRowRDD(shuffleDependency, readMetrics, partitionIndices) + partitionRanges: Option[Array[(Int, Int)]]): ShuffledRowRDD = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionRanges) } def createLocalShuffleRDD( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 4b8c9d88f5b0..86c280918513 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -61,9 +61,8 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA new MapOutputStatistics(index, bytesByPartitionId) } val length = mapOutputStatistics.map(_.bytesByPartitionId.length).head - val validPartitions = (0 until length).toArray val estimatedPartitionStartIndices = - rule.estimatePartitionStartAndEndIndices(mapOutputStatistics, validPartitions).unzip._1 + rule.estimatePartitionStartAndEndIndices(mapOutputStatistics).unzip._1 assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) } @@ -136,7 +135,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA new MapOutputStatistics(0, bytesByPartitionId1), new MapOutputStatistics(1, bytesByPartitionId2)) intercept[AssertionError](rule.estimatePartitionStartAndEndIndices( - mapOutputStatistics, (0 until bytesByPartitionId1.length).toArray)) + mapOutputStatistics, (0 until bytesByPartitionId1.length).toSet)) } { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 238eaaa8f707..5126b2e61192 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -99,7 +99,7 @@ class AdaptiveQueryExecSuite val exchanges = adaptivePlan.collect { case e: Exchange => e } - // assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") + assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") (dfAdaptive.queryExecution.sparkPlan, adaptivePlan) } From 2139f900e74d2c1a29d138eb97d98c6333ab9062 Mon Sep 17 00:00:00 2001 From: jiake Date: Thu, 12 Dec 2019 16:54:28 +0800 Subject: [PATCH 13/23] code style --- .../org/apache/spark/sql/execution/adaptive/QueryStageExec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 44f1f3447f4d..09193633e38a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.adaptive import scala.concurrent.Future + import org.apache.spark.{FutureAction, MapOutputStatistics} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD From 8f12a2b058fed4041527c5b5836ddeb23aa1800e Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 17 Dec 2019 16:29:02 +0800 Subject: [PATCH 14/23] resolve comments --- .../adaptive/OptimizeSkewedPartitions.scala | 78 +---- .../execution/adaptive/QueryStageExec.scala | 24 ++ .../adaptive/ReduceNumShufflePartitions.scala | 30 +- .../adaptive/SkewedShuffledRowRDD.scala | 9 +- .../ReduceNumShufflePartitionsSuite.scala | 5 +- .../adaptive/AdaptiveQueryExecSuite.scala | 314 +++++++++--------- 6 files changed, 200 insertions(+), 260 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index 09572886b5f9..e0fe91c6f0da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -58,19 +58,10 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { 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( + private def getMapStartIndices( stage: QueryStageExec, partitionId: Int, medianSize: Long): Array[Int] = { @@ -91,7 +82,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { /** * 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. + * also depends on which side is skewed. */ private def supportOptimization( joinType: JoinType, @@ -112,7 +103,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { joinType == Inner || joinType == Cross || joinType == RightOuter } - private def getMappersNum(stage: QueryStageExec): Int = { + private def getNumMappers(stage: QueryStageExec): Int = { ShuffleQueryStageExec.getShuffleStage(stage).plan.shuffleDependency.rdd.partitions.length } @@ -142,12 +133,12 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { val isLeftSkew = isSkewed(leftStats, partitionId, leftMedSize) val isRightSkew = isSkewed(rightStats, partitionId, rightMedSize) val leftMapIdStartIndices = if (isLeftSkew && supportSplitOnLeftPartition(joinType)) { - estimateMapStartIndices(left, partitionId, leftMedSize) + getMapStartIndices(left, partitionId, leftMedSize) } else { Array(0) } val rightMapIdStartIndices = if (isRightSkew && supportSplitOnRightPartition(joinType)) { - estimateMapStartIndices(right, partitionId, rightMedSize) + getMapStartIndices(right, partitionId, rightMedSize) } else { Array(0) } @@ -157,12 +148,12 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { for (i <- 0 until leftMapIdStartIndices.length; j <- 0 until rightMapIdStartIndices.length) { val leftEndMapId = if (i == leftMapIdStartIndices.length - 1) { - getMappersNum(left) + getNumMappers(left) } else { leftMapIdStartIndices(i + 1) } val rightEndMapId = if (j == rightMapIdStartIndices.length - 1) { - getMappersNum(right) + getNumMappers(right) } else { rightMapIdStartIndices(j + 1) } @@ -188,9 +179,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { val newStage = ShuffleQueryStageExec.getShuffleStage( sort.child.asInstanceOf[QueryStageExec]).copy( excludedPartitions = skewedPartitions.toSet) - val partialReader = PartialShuffleReader( - newStage, skewedPartitions.toSet) - sort.copy(child = partialReader) + sort.copy(child = newStage) } } subJoins += optimizedSmj @@ -207,8 +196,6 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil - case _: SkewedShufflePartitionReader => Nil - case _: PartialShuffleReader => Nil case _: CoalescedShuffleReaderExec => Nil case stage: ShuffleQueryStageExec => Seq(stage) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => Seq(stage) @@ -217,7 +204,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { val shuffleStages = collectShuffleStages(plan) - if (shuffleStages.length == 2) { + if (shuffleStages.distinct.length == 2) { // Currently we only support handling skewed join for 2 table join. handleSkewJoin(plan) } else { @@ -227,53 +214,6 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } } -/** - * A wrapper of shuffle query stage, which submits one reduce task to read one shuffle partition. - * This is used to handle the non-skewed partitions. - * - * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the - * shuffle exchange node during canonicalization. - * @param excludedPartitions The excluded partitions. - */ -case class PartialShuffleReader( - child: QueryStageExec, excludedPartitions: Set[Int]) extends UnaryExecNode { - override def output: Seq[Attribute] = child.output - - override def doCanonicalize(): SparkPlan = child.canonicalized - - override def outputPartitioning: Partitioning = { - UnknownPartitioning(getPartitionIndexRanges(excludedPartitions).length) - } - - private def getPartitionIndexRanges(omittedPartitions: Set[Int]): Array[(Int, Int)] = { - val length = ShuffleQueryStageExec.getShuffleStage(child) - .plan.shuffleDependency.partitioner.numPartitions - val partitionStartIndices = ArrayBuffer[Int]() - val partitionEndIndices = ArrayBuffer[Int]() - (0 until length).map { i => - if (!omittedPartitions.contains(i)) { - partitionStartIndices += i - partitionEndIndices += i + 1 - } - } - partitionStartIndices.zip(partitionEndIndices).toArray - } - - private var cachedShuffleRDD: ShuffledRowRDD = null - - override protected def doExecute(): RDD[InternalRow] = { - if (cachedShuffleRDD == null) { - cachedShuffleRDD = child match { - case stage: ShuffleQueryStageExec => - stage.plan.createShuffledRDD(Some(getPartitionIndexRanges(excludedPartitions))) - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createShuffledRDD(Some(getPartitionIndexRanges(excludedPartitions))) - } - } - cachedShuffleRDD - } -} - /** * A wrapper of shuffle query stage, which submits one reduce task to read the partition produced * by the mappers in range [startMapId, endMapId]. This is used to handle the skewed partitions. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 09193633e38a..cac582b6fe8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.adaptive +import scala.collection.mutable.ArrayBuffer import scala.concurrent.Future import org.apache.spark.{FutureAction, MapOutputStatistics} @@ -162,6 +163,29 @@ case class ShuffleQueryStageExec( case _ => } } + + private def getPartitionIndexRanges(omittedPartitions: Set[Int]): Array[(Int, Int)] = { + val length = plan.shuffleDependency.partitioner.numPartitions + val partitionStartIndices = ArrayBuffer[Int]() + val partitionEndIndices = ArrayBuffer[Int]() + (0 until length).map { i => + if (!omittedPartitions.contains(i)) { + partitionStartIndices += i + partitionEndIndices += i + 1 + } + } + partitionStartIndices.zip(partitionEndIndices).toArray + } + + private var cachedShuffleRDD: ShuffledRowRDD = null + + override def doExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + cachedShuffleRDD = plan.createShuffledRDD( + Some(getPartitionIndexRanges(excludedPartitions))) + } + cachedShuffleRDD + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index e7162bfee265..6021afcec5f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -65,7 +65,6 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil - case _: SkewedShufflePartitionReader => Nil case stage: ShuffleQueryStageExec => Seq(stage) case _ => plan.children.flatMap(collectShuffleStages) } @@ -90,24 +89,13 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { val distinctNumPreShufflePartitions = validMetrics.map(stats => stats.bytesByPartitionId.length).distinct if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { - val visitedStage = mutable.HashSet[QueryStageExec]() - plan.transformDown { - // even for shuffle exchange whose input RDD has 0 partition, we should still update its - // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same - // number of output partitions. - case stage: ShuffleQueryStageExec if(!visitedStage.contains(stage)) => - visitedStage += stage - val excludedPartitions = - ShuffleQueryStageExec.getShuffleStage(stage).excludedPartitions - val partitionIndices = estimatePartitionStartAndEndIndices( - validMetrics.toArray, excludedPartitions) - visitedStage += stage + val excludedPartitions = + shuffleStages.head.excludedPartitions + val partitionIndices = estimatePartitionStartAndEndIndices( + validMetrics.toArray, excludedPartitions) + plan.transformUp { + case stage: ShuffleQueryStageExec => CoalescedShuffleReaderExec(stage, partitionIndices) - case partialReader: PartialShuffleReader => - visitedStage += partialReader.child - val optimizedPartitionIndices = estimatePartitionStartAndEndIndices( - validMetrics.toArray, partialReader.excludedPartitions) - CoalescedShuffleReaderExec(partialReader.child, optimizedPartitionIndices) } } else { plan @@ -124,7 +112,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { private[sql] def estimatePartitionStartAndEndIndices( mapOutputStatistics: Array[MapOutputStatistics], excludedPartitions: Set[Int] = Set.empty): Array[(Int, Int)] = { - val minNumPostShufflePartitions = conf.minNumPostShufflePartitions + val minNumPostShufflePartitions = conf.minNumPostShufflePartitions - excludedPartitions.size val advisoryTargetPostShuffleInputSize = conf.targetPostShuffleInputSize // If minNumPostShufflePartitions is defined, it is possible that we need to use a // value less than advisoryTargetPostShuffleInputSize as the target input size of @@ -159,13 +147,13 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { val partitionStartIndices = ArrayBuffer[Int]() val partitionEndIndices = ArrayBuffer[Int]() - val numPartitions = mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).head + val numPartitions = distinctNumPreShufflePartitions.head val includedPartitions = (0 until numPartitions).filter(!excludedPartitions.contains(_)) val firstStartIndex = includedPartitions(0) partitionStartIndices += firstStartIndex var postShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(firstStartIndex)).sum var i = firstStartIndex - includedPartitions.filter(_ != firstStartIndex).foreach { + includedPartitions.drop(1).foreach { nextPartitionIndices => var nextShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(nextPartitionIndices)).sum diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala index 412375cadfb9..bb3e445d96bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala @@ -23,14 +23,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} /** - * The [[Partition]] used by [[SkewedShuffledRowRDD]]. A post-shuffle partition - * (identified by `postShufflePartitionIndex`) contains a range of pre-shuffle partitions - * (`preShufflePartitionIndex` from `startMapId` to `endMapId - 1`, inclusive). + * The [[Partition]] used by [[SkewedShuffledRowRDD]]. */ -private final class SkewedShuffledRowRDDPartition( - val postShufflePartitionIndex: Int) extends Partition { - override val index: Int = postShufflePartitionIndex -} +class SkewedShuffledRowRDDPartition(override val index: Int) extends Partition /** * This is a specialized version of [[org.apache.spark.sql.execution.ShuffledRowRDD]]. This is used diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 86c280918513..274fd8fe3867 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -60,7 +60,6 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA case (bytesByPartitionId, index) => new MapOutputStatistics(index, bytesByPartitionId) } - val length = mapOutputStatistics.map(_.bytesByPartitionId.length).head val estimatedPartitionStartIndices = rule.estimatePartitionStartAndEndIndices(mapOutputStatistics).unzip._1 assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) @@ -134,8 +133,8 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA Array( new MapOutputStatistics(0, bytesByPartitionId1), new MapOutputStatistics(1, bytesByPartitionId2)) - intercept[AssertionError](rule.estimatePartitionStartAndEndIndices( - mapOutputStatistics, (0 until bytesByPartitionId1.length).toSet)) + intercept[AssertionError](rule.estimatePartitionStartAndEndIndices( + mapOutputStatistics)) } { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 5126b2e61192..b63a762775c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -38,28 +38,22 @@ class AdaptiveQueryExecSuite import testImplicits._ - protected lazy val skewData1: DataFrame = { - val df1 = - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 5 as key1", "id as value1").toDF() - df1.createOrReplaceTempView("skewData1") - df1 - } - - protected lazy val skewData2: DataFrame = { - val df2 = - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 1 as key2", "id as value2").toDF() - df2.createOrReplaceTempView("skewData2") - df2 - } - protected override def beforeAll(): Unit = { super.beforeAll() - skewData1 - skewData2 + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 2 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + } + + protected override def afterAll(): Unit = { + super.afterAll() + spark.catalog.dropTempView("skewData1") + spark.catalog.dropTempView("skewData2") } setupTestData() @@ -605,153 +599,153 @@ class AdaptiveQueryExecSuite } test("adaptive skew join both in left and right for inner join ") { - withSQLConf( - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR.key -> "1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", - SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "2000") { - val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( - "SELECT * FROM skewData1 join skewData2 ON key1 = key2") - val smj = findTopLevelSortMergeJoin(plan) - assert(smj.size == 1) - // left stats: [4403, 0, 1927, 1927, 1927] - // right stats:[6292, 0, 0, 0, 0] - // the partition 0 in both left and right are all skewed. - // And the map data size of partition 0 in left is - // [539, 539, 490, 405, 405, 405, 405, 405, 405, 405]. - // So split to 10 smjs (the number of mappers). - // and the the map data size of partition 0 in right is - // [955, 593, 593, 593, 593, 593, 593, 593, 593, 593]. - // So split to 10 smjs (the number of mappers). - // So total 101 (10 x 10 + 1) smjs. - // Union - // +- SortMergeJoin - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // . - // . - // . - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - - val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 101) + Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", + SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> reducePostShufflePartitionsEnabled) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 join skewData2 ON key1 = key2") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // the partition 0 in both left and right side are all skewed. + // And divide into 10 splits both in left and right (the number of mappers). + // So there are 10 x 10 smjs for partition 0. + // Partition 4 in left side is skewed and is divided into 10 splits. + // The right side of partition 4 is not skewed. + // So there are 10 smjs for partition 4. + // So total (100 + 10 + 1) smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) + assert(smjAfter.size == 111) + } } } test("adaptive skew join both in left and right for left outer join ") { - withSQLConf( - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR.key -> "1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", - SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "2000") { - val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( - "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") - val smj = findTopLevelSortMergeJoin(plan) - assert(smj.size == 1) - // left stats: [4403, 0, 1927, 1927, 1927] - // right stats:[6292, 0, 0, 0, 0] - // the partition 0 in both left and right are all skewed. - // But for left outer join, we don't split the right partition even skewed. - // And the map data size of partition 0 in left is - // [539, 539, 490, 405, 405, 405, 405, 405, 405, 405]. - // So split to 10 smjs (the number of mappers). - // So total 11 smjs. - // Union - // +- SortMergeJoin - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // . - // . - // . - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - - val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 11) + Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", + SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> reducePostShufflePartitionsEnabled) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // The partition 0 in both left and right are all skewed. + // The partition 4 in left side is skewed. + // But for left outer join, we don't split the right partition even skewed. + // So the partition 0 in left side is divided into 10 splits(the number of mappers). + // the partition 4 in left side is divided into 10 splits(the numer of mappers). + // So total (10 + 10 + 1) smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) + assert(smjAfter.size == 21) + } } } test("adaptive skew join both in left and right for right outer join ") { - withSQLConf( - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR.key -> "1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", - SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "2000") { - val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( - "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") - val smj = findTopLevelSortMergeJoin(plan) - assert(smj.size == 1) - // left stats: [4403, 0, 1927, 1927, 1927] - // right stats:[6292, 0, 0, 0, 0] - // the partition 0 in both left and right are all skewed. - // But for right outer join, we don't split the left partition even skewed. - // And the the map data size of partition 0 in right is - // [955, 593, 593, 593, 593, 593, 593, 593, 593, 593]. - // So split to 10 smjs (the number of mappers) - // So total 11 smjs. - // Union - // +- SortMergeJoin - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // . - // . - // . - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - - val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 11) + Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // The partition 0 in both left and right side are all skewed. + // And the partition 4 in left side is skewed. + // But for right outer join, we don't split the left partition even skewed. + // And divide right side into 10 splits(the number of mappers) + // So total 11 smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) + assert(smjAfter.size == 11) + } } } From 4d256217280cf1d2fac6d48840ac63949b3c8431 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 17 Dec 2019 19:22:42 +0800 Subject: [PATCH 15/23] fix the failed ut --- .../spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index b63a762775c7..3c7076be782d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -51,9 +51,9 @@ class AdaptiveQueryExecSuite } protected override def afterAll(): Unit = { - super.afterAll() spark.catalog.dropTempView("skewData1") spark.catalog.dropTempView("skewData2") + super.afterAll() } setupTestData() From fd9a1aa0e1cc4c9331388e348ab4967c8bf0de28 Mon Sep 17 00:00:00 2001 From: jiake Date: Wed, 1 Jan 2020 09:44:43 +0800 Subject: [PATCH 16/23] resolve the comments and split the mappers based on the map size and max split number --- .../org/apache/spark/MapOutputTracker.scala | 18 +-- .../shuffle/sort/SortShuffleManager.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 8 ++ .../adaptive/OptimizeLocalShuffleReader.scala | 4 +- .../adaptive/OptimizeSkewedPartitions.scala | 114 +++++++++--------- .../execution/adaptive/QueryStageExec.scala | 10 +- .../adaptive/ReduceNumShufflePartitions.scala | 27 +++-- .../adaptive/SkewedShuffledRowRDD.scala | 4 +- .../ReduceNumShufflePartitionsSuite.scala | 4 +- .../adaptive/AdaptiveQueryExecSuite.scala | 85 ++++++------- 10 files changed, 139 insertions(+), 137 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index b65fff6b4020..affaa83a293f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -352,10 +352,10 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging */ def getMapSizesByRange( shuffleId: Int, - startPartition: Int, - endPartition: Int, startMapId: Int, - endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] + endMapId: Int, + startPartition: Int, + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] /** * Deletes map output status information for the specified shuffle stage. @@ -753,10 +753,10 @@ private[spark] class MapOutputTrackerMaster( override def getMapSizesByRange( shuffleId: Int, - startPartition: Int, - endPartition: Int, startMapId: Int, - endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + endMapId: Int, + startPartition: Int, + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => shuffleStatus.withMapStatuses { statuses => @@ -815,10 +815,10 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr override def getMapSizesByRange( shuffleId: Int, - startPartition: Int, - endPartition: Int, startMapId: Int, - endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + endMapId: Int, + startPartition: Int, + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { val statuses = getStatuses(shuffleId, conf) try { MapOutputTracker.convertMapStatuses( diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index e62f4a54132a..04b59ddcc406 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -140,7 +140,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByRange( - handle.shuffleId, startPartition, endPartition, startMapId, endMapId) + handle.shuffleId, startMapId, endMapId, startPartition, endPartition) new BlockStoreShuffleReader( handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 458d2b62b48c..c1e2e46b18b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -473,6 +473,14 @@ object SQLConf { .intConf .createWithDefault(10) + val ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS = + buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionMaxSplits") + .doc("Configures the maximum number of task to handle a skewed partition in adaptive skewed" + + "join.") + .intConf + .checkValue( _ >= 1, "The split size at least be 1") + .createWithDefault(5) + val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") .doc("The relation with a non-empty partition ratio lower than this config will not be " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index fd55f6c5eb4e..72d74818e8a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -165,9 +165,9 @@ case class LocalShuffleReaderExec( // before shuffle. if (partitionStartIndicesPerMapper.forall(_.length == 1)) { child match { - case ShuffleQueryStageExec(_, s: ShuffleExchangeExec) => + case ShuffleQueryStageExec(_, s: ShuffleExchangeExec, _) => s.child.outputPartitioning - case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeExec)) => + case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeExec), _) => s.child.outputPartitioning match { case e: Expression => r.updateAttr(e).asInstanceOf[Partitioning] case other => other diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index e0fe91c6f0da..cb89a54c6ed7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -59,39 +59,47 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } /** - * Split the partition into the number of mappers. Each split read data from each mapper. + * Get the map size of the specific reduce shuffle Id. */ - private def getMapStartIndices( - 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) + private def getMapSizeForReduceId(partitionId: Int, shuffleId: Int): Array[Long] = { + val mapOutputTracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + mapOutputTracker.shuffleStatuses.get(shuffleId). + get.mapStatuses.map{_.getSizeForBlock(partitionId)} } /** - * 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 which side is skewed. + * Split the skewed partition based on the map size and the max split number. */ - private def supportOptimization( - joinType: JoinType, - leftStage: QueryStageExec, - rightStage: QueryStageExec): Boolean = { - val joinTypeSupported = supportedJoinTypes.contains(joinType) - val shuffleStageCheck = ShuffleQueryStageExec.isShuffleQueryStageExec(leftStage) && - ShuffleQueryStageExec.isShuffleQueryStageExec(rightStage) - joinTypeSupported && shuffleStageCheck + private def getMapStartIndices(stage: ShuffleQueryStageExec, partitionId: Int): Array[Int] = { + val shuffleId = stage.shuffle.shuffleDependency.shuffleHandle.shuffleId + val mapPartitionSize = getMapSizeForReduceId(partitionId, shuffleId) + val numSplits = math.min(conf.getConf( + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS), mapPartitionSize.length) + val avgPartitionSize = mapPartitionSize.sum / numSplits + val partitionIndices = (0 until mapPartitionSize.length) + val partitionStartIndices = ArrayBuffer[Int]() + var postMapPartitionSize = mapPartitionSize(0) + partitionStartIndices += 0 + partitionIndices.drop(1).foreach { + nextPartitionIndex => + var nextMapPartitionSize = mapPartitionSize(nextPartitionIndex) + if (postMapPartitionSize + nextMapPartitionSize > avgPartitionSize) { + partitionStartIndices += nextPartitionIndex + postMapPartitionSize = nextMapPartitionSize + } else { + postMapPartitionSize += nextMapPartitionSize + } + } + + if (partitionStartIndices.size > numSplits) { + partitionStartIndices.take(numSplits).toArray + } else partitionStartIndices.toArray + } + + private def getStatistics(stage: ShuffleQueryStageExec): MapOutputStatistics = { + assert(stage.resultOption.isDefined, "ShuffleQueryStageExec should" + + " already be ready when executing OptimizeSkewedPartitions rule") + stage.resultOption.get.asInstanceOf[MapOutputStatistics] } private def supportSplitOnLeftPartition(joinType: JoinType) = { @@ -103,28 +111,29 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { joinType == Inner || joinType == Cross || joinType == RightOuter } - private def getNumMappers(stage: QueryStageExec): Int = { - ShuffleQueryStageExec.getShuffleStage(stage).plan.shuffleDependency.rdd.partitions.length + private def getNumMappers(stage: ShuffleQueryStageExec): Int = { + stage.shuffle.shuffleDependency.rdd.partitions.length } def handleSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { case smj @ SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, - SortExec(_, _, left: QueryStageExec, _), - SortExec(_, _, right: QueryStageExec, _)) - if supportOptimization(joinType, left, right) => + SortExec(_, _, left: ShuffleQueryStageExec, _), + SortExec(_, _, right: ShuffleQueryStageExec, _)) + if supportedJoinTypes.contains(joinType) => val leftStats = getStatistics(left) val rightStats = getStatistics(right) val numPartitions = leftStats.bytesByPartitionId.length val leftMedSize = medianSize(leftStats) val rightMedSize = medianSize(rightStats) + val leftSizeInfo = s"median size: $leftMedSize, max size: ${leftStats.bytesByPartitionId.max}" + val rightSizeInfo = s"median size: $rightMedSize," + + s" max size: ${rightStats.bytesByPartitionId.max}" logDebug( s""" |Try to optimize skewed join. - |Left side partition size: median size: $leftMedSize, - | max size: ${leftStats.bytesByPartitionId.max} - |Right side partition size: median size: $rightMedSize, - | max size: ${rightStats.bytesByPartitionId.max} + |Left side partition size: $leftSizeInfo + |Right side partition size: $rightSizeInfo """.stripMargin) val skewedPartitions = mutable.HashSet[Int]() @@ -133,12 +142,12 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { val isLeftSkew = isSkewed(leftStats, partitionId, leftMedSize) val isRightSkew = isSkewed(rightStats, partitionId, rightMedSize) val leftMapIdStartIndices = if (isLeftSkew && supportSplitOnLeftPartition(joinType)) { - getMapStartIndices(left, partitionId, leftMedSize) + getMapStartIndices(left, partitionId) } else { Array(0) } val rightMapIdStartIndices = if (isRightSkew && supportSplitOnRightPartition(joinType)) { - getMapStartIndices(right, partitionId, rightMedSize) + getMapStartIndices(right, partitionId) } else { Array(0) } @@ -159,12 +168,10 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } // TODO: we may can optimize the sort merge join to broad cast join after // obtaining the raw data size of per partition, - val leftSkewedReader = - SkewedShufflePartitionReader( + val leftSkewedReader = SkewedShufflePartitionReader( left, partitionId, leftMapIdStartIndices(i), leftEndMapId) - val rightSkewedReader = - SkewedShufflePartitionReader(right, partitionId, + val rightSkewedReader = SkewedShufflePartitionReader(right, partitionId, rightMapIdStartIndices(j), rightEndMapId) subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, leftSkewedReader, rightSkewedReader) @@ -174,13 +181,12 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { logDebug(s"number of skewed partitions is ${skewedPartitions.size}") if (skewedPartitions.size > 0) { val optimizedSmj = smj.transformDown { - case sort: SortExec if ( - ShuffleQueryStageExec.isShuffleQueryStageExec(sort.child)) => { - val newStage = ShuffleQueryStageExec.getShuffleStage( - sort.child.asInstanceOf[QueryStageExec]).copy( + case sort: SortExec if (sort.child.isInstanceOf[ShuffleQueryStageExec]) => + val shuffleStage = sort.child.asInstanceOf[ShuffleQueryStageExec] + val newStage = shuffleStage.copy( excludedPartitions = skewedPartitions.toSet) + newStage.resultOption = shuffleStage.resultOption sort.copy(child = newStage) - } } subJoins += optimizedSmj UnionExec(subJoins) @@ -198,7 +204,6 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { case _: LocalShuffleReaderExec => Nil case _: CoalescedShuffleReaderExec => Nil case stage: ShuffleQueryStageExec => Seq(stage) - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => Seq(stage) case _ => plan.children.flatMap(collectShuffleStages) } @@ -215,8 +220,9 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } /** - * A wrapper of shuffle query stage, which submits one reduce task to read the partition produced - * by the mappers in range [startMapId, endMapId]. This is used to handle the skewed partitions. + * A wrapper of shuffle query stage, which submits one reduce task to read a single + * shuffle partition 'partitionIndex' produced by the mappers in range [startMapId, endMapId). + * This is used to handle the skewed partitions. * * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the * shuffle exchange node during canonicalization. @@ -243,9 +249,9 @@ case class SkewedShufflePartitionReader( if (cachedSkewedShuffleRDD == null) { cachedSkewedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.plan.createSkewedShuffleRDD(partitionIndex, startMapId, endMapId) - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createSkewedShuffleRDD(partitionIndex, startMapId, endMapId) + stage.shuffle.createSkewedShuffleRDD(partitionIndex, startMapId, endMapId) + case _ => + throw new IllegalStateException("operating on canonicalization plan") } } cachedSkewedShuffleRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index cac582b6fe8e..80487c62515c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -164,12 +164,12 @@ case class ShuffleQueryStageExec( } } - private def getPartitionIndexRanges(omittedPartitions: Set[Int]): Array[(Int, Int)] = { - val length = plan.shuffleDependency.partitioner.numPartitions + private def getPartitionIndexRanges(): Array[(Int, Int)] = { + val length = shuffle.shuffleDependency.partitioner.numPartitions val partitionStartIndices = ArrayBuffer[Int]() val partitionEndIndices = ArrayBuffer[Int]() (0 until length).map { i => - if (!omittedPartitions.contains(i)) { + if (!excludedPartitions.contains(i)) { partitionStartIndices += i partitionEndIndices += i + 1 } @@ -181,8 +181,8 @@ case class ShuffleQueryStageExec( override def doExecute(): RDD[InternalRow] = { if (cachedShuffleRDD == null) { - cachedShuffleRDD = plan.createShuffledRDD( - Some(getPartitionIndexRanges(excludedPartitions))) + cachedShuffleRDD = shuffle.createShuffledRDD( + Some(getPartitionIndexRanges())) } cachedShuffleRDD } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 6021afcec5f4..7c9387545e8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -28,8 +28,6 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} import org.apache.spark.sql.internal.SQLConf -import scala.collection.mutable - /** * A rule to adjust the post shuffle partitions based on the map output statistics. * @@ -56,7 +54,9 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { if (!conf.reducePostShufflePartitionsEnabled) { return plan } - if (!plan.collectLeaves().forall(_.isInstanceOf[QueryStageExec])) { + // we ned skip the leaf node of 'SkewedShufflePartitionReader' + val leafNodes = plan.collectLeaves().filter(!_.isInstanceOf[SkewedShufflePartitionReader]) + if (!leafNodes.forall(_.isInstanceOf[QueryStageExec])) { // If not all leaf nodes are query stages, it's not safe to reduce the number of // shuffle partitions, because we may break the assumption that all children of a spark plan // have same number of output partitions. @@ -88,12 +88,17 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // partition) and a result of a SortMergeJoin (multiple partitions). val distinctNumPreShufflePartitions = validMetrics.map(stats => stats.bytesByPartitionId.length).distinct - if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { - val excludedPartitions = - shuffleStages.head.excludedPartitions + val distinctExcludedPartitions = shuffleStages.map(_.excludedPartitions).distinct + if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1 + && distinctExcludedPartitions.length == 1) { + val excludedPartitions = shuffleStages.head.excludedPartitions val partitionIndices = estimatePartitionStartAndEndIndices( validMetrics.toArray, excludedPartitions) + // This transformation adds new nodes, so we must use `transformUp` here. plan.transformUp { + // even for shuffle exchange whose input RDD has 0 partition, we should still update its + // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same + // number of output partitions. case stage: ShuffleQueryStageExec => CoalescedShuffleReaderExec(stage, partitionIndices) } @@ -154,19 +159,19 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { var postShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(firstStartIndex)).sum var i = firstStartIndex includedPartitions.drop(1).foreach { - nextPartitionIndices => + nextPartitionIndex => var nextShuffleInputSize = - mapOutputStatistics.map(_.bytesByPartitionId(nextPartitionIndices)).sum + mapOutputStatistics.map(_.bytesByPartitionId(nextPartitionIndex)).sum // If nextPartitionIndices is skewed and omitted, or including // the nextShuffleInputSize would exceed the target partition size, // then start a new partition. - if (nextPartitionIndices != i + 1 || + if (nextPartitionIndex != i + 1 || (postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize)) { partitionEndIndices += i + 1 - partitionStartIndices += nextPartitionIndices + partitionStartIndices += nextPartitionIndex // reset postShuffleInputSize. postShuffleInputSize = nextShuffleInputSize - i = nextPartitionIndices + i = nextPartitionIndex } else { postShuffleInputSize += nextShuffleInputSize i += 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala index bb3e445d96bc..2dbe6614251b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala @@ -46,9 +46,7 @@ class SkewedShuffledRowRDD( override def getDependencies: Seq[Dependency[_]] = List(dependency) override def getPartitions: Array[Partition] = { - Array.tabulate[Partition](1) { i => - new SkewedShuffledRowRDDPartition(i) - } + Array(new SkewedShuffledRowRDDPartition(0)) } override def getPreferredLocations(partition: Partition): Seq[String] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 274fd8fe3867..d7f303ae0e10 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -533,7 +533,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val finalPlan = resultDf.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan assert(finalPlan.collect { - case ShuffleQueryStageExec(_, r: ReusedExchangeExec) => r + case ShuffleQueryStageExec(_, r: ReusedExchangeExec, _) => r }.length == 2) assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 3) @@ -566,7 +566,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val reusedStages = level1Stages.flatMap { stage => stage.plan.collect { - case ShuffleQueryStageExec(_, r: ReusedExchangeExec) => r + case ShuffleQueryStageExec(_, r: ReusedExchangeExec, _) => r } } assert(reusedStages.length == 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 3c7076be782d..6f98d102f709 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -21,7 +21,6 @@ import java.io.File import java.net.URI import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} -import org.apache.spark.sql.DataFrame import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec} @@ -111,7 +110,7 @@ class AdaptiveQueryExecSuite private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = { collectInPlanAndSubqueries(plan) { - case ShuffleQueryStageExec(_, e: ReusedExchangeExec) => e + case ShuffleQueryStageExec(_, e: ReusedExchangeExec, _) => e case BroadcastQueryStageExec(_, e: ReusedExchangeExec) => e } } @@ -598,26 +597,28 @@ class AdaptiveQueryExecSuite } } - test("adaptive skew join both in left and right for inner join ") { + test("adaptive skew join both in left and right for inner join," + + " left outer join and right outer join") { Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> reducePostShufflePartitionsEnabled) { - val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + // skewed inner join optimization + val (innerPlan, innerAdaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM skewData1 join skewData2 ON key1 = key2") - val smj = findTopLevelSortMergeJoin(plan) - assert(smj.size == 1) + val innerSmj = findTopLevelSortMergeJoin(innerPlan) + assert(innerSmj.size == 1) // left stats: [3496, 0, 0, 0, 4014] // right stats:[6292, 0, 0, 0, 0] // the partition 0 in both left and right side are all skewed. - // And divide into 10 splits both in left and right (the number of mappers). - // So there are 10 x 10 smjs for partition 0. - // Partition 4 in left side is skewed and is divided into 10 splits. + // And divide into 5 splits both in left and right (the max splits number). + // So there are 5 x 5 smjs for partition 0. + // Partition 4 in left side is skewed and is divided into 5 splits. // The right side of partition 4 is not skewed. - // So there are 10 smjs for partition 4. - // So total (100 + 10 + 1) smjs. + // So there are 5 smjs for partition 4. + // So total (25 + 5 + 1) smjs. // Union // +- SortMergeJoin // +- Sort @@ -625,7 +626,7 @@ class AdaptiveQueryExecSuite // +- ShuffleQueryStage // +- Sort // +- CoalescedShuffleReader - // +- ShuffleQueryStage + // +- ShuffleQueryStage // +- SortMergeJoin // +- Sort // +- SkewedShuffleReader @@ -644,31 +645,22 @@ class AdaptiveQueryExecSuite // +- SkewedShuffleReader // +- ShuffleQueryStage - val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 111) - } - } - } + val innerSmjAfter = findTopLevelSortMergeJoin(innerAdaptivePlan) + assert(innerSmjAfter.size == 31) - test("adaptive skew join both in left and right for left outer join ") { - Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => - withSQLConf( - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", - SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> reducePostShufflePartitionsEnabled) { - val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + // skewed left outer join optimization + val (leftPlan, leftAdaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") - val smj = findTopLevelSortMergeJoin(plan) - assert(smj.size == 1) + val leftSmj = findTopLevelSortMergeJoin(leftPlan) + assert(leftSmj.size == 1) // left stats: [3496, 0, 0, 0, 4014] // right stats:[6292, 0, 0, 0, 0] // The partition 0 in both left and right are all skewed. // The partition 4 in left side is skewed. // But for left outer join, we don't split the right partition even skewed. - // So the partition 0 in left side is divided into 10 splits(the number of mappers). - // the partition 4 in left side is divided into 10 splits(the numer of mappers). - // So total (10 + 10 + 1) smjs. + // So the partition 0 in left side is divided into 5 splits(the max split number). + // the partition 4 in left side is divided into 5 splits(the max split number). + // So total (5 + 5 + 1) smjs. // Union // +- SortMergeJoin // +- Sort @@ -676,7 +668,7 @@ class AdaptiveQueryExecSuite // +- ShuffleQueryStage // +- Sort // +- CoalescedShuffleReader - // +- ShuffleQueryStage + // +- ShuffleQueryStage // +- SortMergeJoin // +- Sort // +- SkewedShuffleReader @@ -695,28 +687,21 @@ class AdaptiveQueryExecSuite // +- SkewedShuffleReader // +- ShuffleQueryStage - val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 21) - } - } - } - test("adaptive skew join both in left and right for right outer join ") { - Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => - withSQLConf( - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100") { - val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + val leftSmjAfter = findTopLevelSortMergeJoin(leftAdaptivePlan) + assert(leftSmjAfter.size == 11) + + // skewed right outer join optimization + val (rightPlan, rightAdaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") - val smj = findTopLevelSortMergeJoin(plan) - assert(smj.size == 1) + val rightSmj = findTopLevelSortMergeJoin(rightPlan) + assert(rightSmj.size == 1) // left stats: [3496, 0, 0, 0, 4014] // right stats:[6292, 0, 0, 0, 0] // The partition 0 in both left and right side are all skewed. // And the partition 4 in left side is skewed. // But for right outer join, we don't split the left partition even skewed. - // And divide right side into 10 splits(the number of mappers) - // So total 11 smjs. + // And divide right side into 5 splits(the max split number) + // So total 6 smjs. // Union // +- SortMergeJoin // +- Sort @@ -724,7 +709,7 @@ class AdaptiveQueryExecSuite // +- ShuffleQueryStage // +- Sort // +- CoalescedShuffleReader - // +- ShuffleQueryStage + // +- ShuffleQueryStage // +- SortMergeJoin // +- Sort // +- SkewedShuffleReader @@ -743,8 +728,8 @@ class AdaptiveQueryExecSuite // +- SkewedShuffleReader // +- ShuffleQueryStage - val smjAfter = findTopLevelSortMergeJoin(adaptivePlan) - assert(smjAfter.size == 11) + val rightSmjAfter = findTopLevelSortMergeJoin(rightAdaptivePlan) + assert(rightSmjAfter.size == 6) } } } From cac6ca86031da9e8bdbba88ac111884c1cc17e42 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 7 Jan 2020 16:13:04 +0800 Subject: [PATCH 17/23] resolve the comments --- .../org/apache/spark/MapOutputTracker.scala | 26 +++++++------- .../spark/sql/execution/ShuffledRowRDD.scala | 2 +- .../adaptive/OptimizeSkewedPartitions.scala | 34 +++++++++---------- .../execution/adaptive/QueryStageExec.scala | 11 ++++-- .../adaptive/ReduceNumShufflePartitions.scala | 7 ++-- .../ReduceNumShufflePartitionsSuite.scala | 2 +- .../adaptive/AdaptiveQueryExecSuite.scala | 34 ++++++++----------- 7 files changed, 56 insertions(+), 60 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index affaa83a293f..12b53e3736b4 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -352,8 +352,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging */ def getMapSizesByRange( shuffleId: Int, - startMapId: Int, - endMapId: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] @@ -753,15 +753,15 @@ private[spark] class MapOutputTrackerMaster( override def getMapSizesByRange( shuffleId: Int, - startMapId: Int, - endMapId: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => shuffleStatus.withMapStatuses { statuses => MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses, startMapId, endMapId) + shuffleId, startPartition, endPartition, statuses, startMapIndex, endMapIndex) } case None => Iterator.empty @@ -815,14 +815,14 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr override def getMapSizesByRange( shuffleId: Int, - startMapId: Int, - endMapId: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { val statuses = getStatuses(shuffleId, conf) try { MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses, startMapId, endMapId) + shuffleId, startPartition, endPartition, statuses, startMapIndex, endMapIndex) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -981,8 +981,8 @@ private[spark] object MapOutputTracker extends Logging { * @param startPartition Start of map output partition ID range (included in range) * @param endPartition End of map output partition ID range (excluded from range) * @param statuses List of map statuses, indexed by map partition index. - * @param startMapId Start Map ID. - * @param endMapId End Map ID. + * @param startMapIndex Start Map index. + * @param endMapIndex End Map index. * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. @@ -992,12 +992,12 @@ private[spark] object MapOutputTracker extends Logging { startPartition: Int, endPartition: Int, statuses: Array[MapStatus], - startMapId : Int, - endMapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + startMapIndex : Int, + endMapIndex: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { assert (statuses != null) val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] val iter = statuses.iterator.zipWithIndex - for ((status, mapIndex) <- iter.slice(startMapId, endMapId)) { + for ((status, mapIndex) <- iter.slice(startMapIndex, endMapIndex)) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" logError(errorMessage) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index cf07638ceea5..efa493923ccc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -127,7 +127,7 @@ class ShuffledRowRDD( private[this] val numPreShufflePartitions = dependency.partitioner.numPartitions private[this] val partitionStartIndices: Array[Int] = specifiedPartitionIndices match { - case Some(indices) => indices.unzip._1 + case Some(indices) => indices.map(_._1) case None => // When specifiedPartitionStartIndices is not defined, every post-shuffle partition // corresponds to a pre-shuffle partition. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index cb89a54c6ed7..c80a85cb19b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -61,10 +61,9 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { /** * Get the map size of the specific reduce shuffle Id. */ - private def getMapSizeForReduceId(partitionId: Int, shuffleId: Int): Array[Long] = { + private def getMapSizesForReduceId(shuffleId: Int, partitionId: Int): Array[Long] = { val mapOutputTracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - mapOutputTracker.shuffleStatuses.get(shuffleId). - get.mapStatuses.map{_.getSizeForBlock(partitionId)} + mapOutputTracker.shuffleStatuses(shuffleId).mapStatuses.map{_.getSizeForBlock(partitionId)} } /** @@ -72,18 +71,19 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { */ private def getMapStartIndices(stage: ShuffleQueryStageExec, partitionId: Int): Array[Int] = { val shuffleId = stage.shuffle.shuffleDependency.shuffleHandle.shuffleId - val mapPartitionSize = getMapSizeForReduceId(partitionId, shuffleId) + val mapPartitionSizes = getMapSizesForReduceId(shuffleId, partitionId) val numSplits = math.min(conf.getConf( - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS), mapPartitionSize.length) - val avgPartitionSize = mapPartitionSize.sum / numSplits - val partitionIndices = (0 until mapPartitionSize.length) + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS), mapPartitionSizes.length) + val avgPartitionSize = mapPartitionSizes.sum / numSplits + val advisoryPartitionSize = math.max(avgPartitionSize, + conf.getConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE)) + val partitionIndices = mapPartitionSizes.indices val partitionStartIndices = ArrayBuffer[Int]() - var postMapPartitionSize = mapPartitionSize(0) + var postMapPartitionSize = mapPartitionSizes(0) partitionStartIndices += 0 - partitionIndices.drop(1).foreach { - nextPartitionIndex => - var nextMapPartitionSize = mapPartitionSize(nextPartitionIndex) - if (postMapPartitionSize + nextMapPartitionSize > avgPartitionSize) { + partitionIndices.drop(1).foreach { nextPartitionIndex => + var nextMapPartitionSize = mapPartitionSizes(nextPartitionIndex) + if (postMapPartitionSize + nextMapPartitionSize > advisoryPartitionSize) { partitionStartIndices += nextPartitionIndex postMapPartitionSize = nextMapPartitionSize } else { @@ -181,7 +181,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { logDebug(s"number of skewed partitions is ${skewedPartitions.size}") if (skewedPartitions.size > 0) { val optimizedSmj = smj.transformDown { - case sort: SortExec if (sort.child.isInstanceOf[ShuffleQueryStageExec]) => + case sort @ SortExec(_, _, shuffleStage: ShuffleQueryStageExec, _) => val shuffleStage = sort.child.asInstanceOf[ShuffleQueryStageExec] val newStage = shuffleStage.copy( excludedPartitions = skewedPartitions.toSet) @@ -209,7 +209,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { val shuffleStages = collectShuffleStages(plan) - if (shuffleStages.distinct.length == 2) { + if (shuffleStages.length == 2) { // Currently we only support handling skewed join for 2 table join. handleSkewJoin(plan) } else { @@ -224,8 +224,8 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { * shuffle partition 'partitionIndex' produced by the mappers in range [startMapId, endMapId). * This is used to handle the skewed partitions. * - * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the - * shuffle exchange node during canonicalization. + * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange + * node during canonicalization. * @param partitionIndex The pre shuffle partition index. * @param startMapId The start map id. * @param endMapId The end map id. @@ -238,8 +238,6 @@ case class SkewedShufflePartitionReader( override def output: Seq[Attribute] = child.output - override def doCanonicalize(): SparkPlan = child.canonicalized - override def outputPartitioning: Partitioning = { UnknownPartitioning(1) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 80487c62515c..349f2037b773 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -177,12 +177,17 @@ case class ShuffleQueryStageExec( partitionStartIndices.zip(partitionEndIndices).toArray } - private var cachedShuffleRDD: ShuffledRowRDD = null + private var cachedShuffleRDD: RDD[InternalRow] = null override def doExecute(): RDD[InternalRow] = { if (cachedShuffleRDD == null) { - cachedShuffleRDD = shuffle.createShuffledRDD( - Some(getPartitionIndexRanges())) + cachedShuffleRDD = excludedPartitions match { + case e if e.isEmpty => + plan.execute() + case _ => + shuffle.createShuffledRDD( + Some(getPartitionIndexRanges())) + } } cachedShuffleRDD } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 7c9387545e8b..ed8769c99c32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -54,7 +54,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { if (!conf.reducePostShufflePartitionsEnabled) { return plan } - // we ned skip the leaf node of 'SkewedShufflePartitionReader' + // we need skip the leaf node of 'SkewedShufflePartitionReader' val leafNodes = plan.collectLeaves().filter(!_.isInstanceOf[SkewedShufflePartitionReader]) if (!leafNodes.forall(_.isInstanceOf[QueryStageExec])) { // If not all leaf nodes are query stages, it's not safe to reduce the number of @@ -158,9 +158,8 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { partitionStartIndices += firstStartIndex var postShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(firstStartIndex)).sum var i = firstStartIndex - includedPartitions.drop(1).foreach { - nextPartitionIndex => - var nextShuffleInputSize = + includedPartitions.drop(1).foreach { nextPartitionIndex => + val nextShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(nextPartitionIndex)).sum // If nextPartitionIndices is skewed and omitted, or including // the nextShuffleInputSize would exceed the target partition size, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index d7f303ae0e10..463b023cd3c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -61,7 +61,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA new MapOutputStatistics(index, bytesByPartitionId) } val estimatedPartitionStartIndices = - rule.estimatePartitionStartAndEndIndices(mapOutputStatistics).unzip._1 + rule.estimatePartitionStartAndEndIndices(mapOutputStatistics).map(_._1) assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 6f98d102f709..4dc735b58f24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -37,24 +37,6 @@ class AdaptiveQueryExecSuite import testImplicits._ - protected override def beforeAll(): Unit = { - super.beforeAll() - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 2 as key1", "id as value1") - .createOrReplaceTempView("skewData1") - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 1 as key2", "id as value2") - .createOrReplaceTempView("skewData2") - } - - protected override def afterAll(): Unit = { - spark.catalog.dropTempView("skewData1") - spark.catalog.dropTempView("skewData2") - super.afterAll() - } - setupTestData() private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { @@ -597,14 +579,24 @@ class AdaptiveQueryExecSuite } } - test("adaptive skew join both in left and right for inner join," + + test("SPARK-29544: adaptive skew join both in left and right for inner join," + " left outer join and right outer join") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 2 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", - SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> reducePostShufflePartitionsEnabled) { + SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> reducePostShufflePartitionsEnabled, + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "700") { // skewed inner join optimization val (innerPlan, innerAdaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM skewData1 join skewData2 ON key1 = key2") @@ -732,6 +724,8 @@ class AdaptiveQueryExecSuite assert(rightSmjAfter.size == 6) } } + spark.catalog.dropTempView("skewData1") + spark.catalog.dropTempView("skewData2") } test("SPARK-30291: AQE should catch the exceptions when doing materialize") { From b464b22f09d907b77cd8ec077ae3df919528b8da Mon Sep 17 00:00:00 2001 From: jiake Date: Wed, 8 Jan 2020 18:24:31 +0800 Subject: [PATCH 18/23] resolve the comments --- .../sql/execution/adaptive/OptimizeSkewedPartitions.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index c80a85cb19b0..f7a4bb574c3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -19,7 +19,6 @@ 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 @@ -31,7 +30,6 @@ 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] { @@ -179,7 +177,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } } logDebug(s"number of skewed partitions is ${skewedPartitions.size}") - if (skewedPartitions.size > 0) { + if (skewedPartitions.nonEmpty) { val optimizedSmj = smj.transformDown { case sort @ SortExec(_, _, shuffleStage: ShuffleQueryStageExec, _) => val shuffleStage = sort.child.asInstanceOf[ShuffleQueryStageExec] From 62bb9508c38979278266d29843c8bc816dd2cf3f Mon Sep 17 00:00:00 2001 From: jiake Date: Wed, 8 Jan 2020 21:59:16 +0800 Subject: [PATCH 19/23] resolve the comments --- .../scala/org/apache/spark/MapOutputTracker.scala | 15 ++++++++------- .../org/apache/spark/shuffle/ShuffleManager.scala | 6 +++--- .../spark/shuffle/sort/SortShuffleManager.scala | 6 +++--- .../adaptive/OptimizeSkewedPartitions.scala | 10 +++++----- 4 files changed, 19 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 12b53e3736b4..b6396c7b03fe 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -344,7 +344,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * Called from executors to get the server URIs and output sizes for each shuffle block that * needs to be read from a given range of map output partitions (startPartition is included but * endPartition is excluded from the range) and is produced by - * a range of mappers (startMapId, endMapId, startMapId is included and the endMapId is excluded). + * a range of mappers (startMapIndex, endMapIndex, startMapIndex is included and + * the endMapIndex is excluded). * * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) @@ -694,20 +695,20 @@ private[spark] class MapOutputTrackerMaster( * executor id on that host. * * @param dep shuffle dependency object - * @param startMapId the start map id - * @param endMapId the end map id + * @param startMapIndex the start map index + * @param endMapIndex the end map index * @return a sequence of locations where task runs. */ def getMapLocation( dep: ShuffleDependency[_, _, _], - startMapId: Int, - endMapId: Int): Seq[String] = + startMapIndex: Int, + endMapIndex: Int): Seq[String] = { val shuffleStatus = shuffleStatuses.get(dep.shuffleId).orNull if (shuffleStatus != null) { shuffleStatus.withMapStatuses { statuses => - if (startMapId < endMapId && (startMapId >= 0 && endMapId < statuses.length)) { - val statusesPicked = statuses.slice(startMapId, endMapId).filter(_ != null) + if (startMapIndex < endMapIndex && (startMapIndex >= 0 && endMapIndex < statuses.length)) { + val statusesPicked = statuses.slice(startMapIndex, endMapIndex).filter(_ != null) statusesPicked.map(_.location.host).toSeq } else { Nil diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index bbeb10f6de49..057b0d6e0b0a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -56,13 +56,13 @@ private[spark] trait ShuffleManager { /** * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to - * read from map output (startMapId to endMapId - 1, inclusive). + * read from map output (startMapIndex to endMapIndex - 1, inclusive). * Called on executors by reduce tasks. */ def getReaderForRange[K, C]( handle: ShuffleHandle, - startMapId: Int, - endMapId: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int, context: TaskContext, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 04b59ddcc406..aefcb59b8bb8 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -133,14 +133,14 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager override def getReaderForRange[K, C]( handle: ShuffleHandle, - startMapId: Int, - endMapId: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByRange( - handle.shuffleId, startMapId, endMapId, startPartition, endPartition) + handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) new BlockStoreShuffleReader( handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index f7a4bb574c3e..e89453c0eaa4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -70,9 +70,9 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { private def getMapStartIndices(stage: ShuffleQueryStageExec, partitionId: Int): Array[Int] = { val shuffleId = stage.shuffle.shuffleDependency.shuffleHandle.shuffleId val mapPartitionSizes = getMapSizesForReduceId(shuffleId, partitionId) - val numSplits = math.min(conf.getConf( + val maxSplits = math.min(conf.getConf( SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS), mapPartitionSizes.length) - val avgPartitionSize = mapPartitionSizes.sum / numSplits + val avgPartitionSize = mapPartitionSizes.sum / maxSplits val advisoryPartitionSize = math.max(avgPartitionSize, conf.getConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE)) val partitionIndices = mapPartitionSizes.indices @@ -80,7 +80,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { var postMapPartitionSize = mapPartitionSizes(0) partitionStartIndices += 0 partitionIndices.drop(1).foreach { nextPartitionIndex => - var nextMapPartitionSize = mapPartitionSizes(nextPartitionIndex) + val nextMapPartitionSize = mapPartitionSizes(nextPartitionIndex) if (postMapPartitionSize + nextMapPartitionSize > advisoryPartitionSize) { partitionStartIndices += nextPartitionIndex postMapPartitionSize = nextMapPartitionSize @@ -89,8 +89,8 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } } - if (partitionStartIndices.size > numSplits) { - partitionStartIndices.take(numSplits).toArray + if (partitionStartIndices.size > maxSplits) { + partitionStartIndices.take(maxSplits).toArray } else partitionStartIndices.toArray } From a0e03b66fcd1aa838feec58ef918d853ce78c71d Mon Sep 17 00:00:00 2001 From: jiake Date: Thu, 9 Jan 2020 14:34:33 +0800 Subject: [PATCH 20/23] resolve the comments --- .../org/apache/spark/MapOutputTracker.scala | 2 + .../adaptive/OptimizeSkewedPartitions.scala | 27 +- .../execution/adaptive/QueryStageExec.scala | 10 +- .../adaptive/SkewedShuffledRowRDD.scala | 14 +- .../exchange/ShuffleExchangeExec.scala | 7 +- .../adaptive/AdaptiveQueryExecSuite.scala | 274 +++++++++--------- 6 files changed, 163 insertions(+), 171 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index b6396c7b03fe..5506a4836e67 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -758,6 +758,8 @@ private[spark] class MapOutputTrackerMaster( endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + logDebug(s"Fetching outputs for shuffle $shuffleId, mappers $startMapIndex - $endMapIndex" + + s"partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => shuffleStatus.withMapStatuses { statuses => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index e89453c0eaa4..5b2e5e0a9701 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -80,13 +80,13 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { var postMapPartitionSize = mapPartitionSizes(0) partitionStartIndices += 0 partitionIndices.drop(1).foreach { nextPartitionIndex => - val nextMapPartitionSize = mapPartitionSizes(nextPartitionIndex) - if (postMapPartitionSize + nextMapPartitionSize > advisoryPartitionSize) { - partitionStartIndices += nextPartitionIndex - postMapPartitionSize = nextMapPartitionSize - } else { - postMapPartitionSize += nextMapPartitionSize - } + val nextMapPartitionSize = mapPartitionSizes(nextPartitionIndex) + if (postMapPartitionSize + nextMapPartitionSize > advisoryPartitionSize) { + partitionStartIndices += nextPartitionIndex + postMapPartitionSize = nextMapPartitionSize + } else { + postMapPartitionSize += nextMapPartitionSize + } } if (partitionStartIndices.size > maxSplits) { @@ -180,7 +180,6 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { if (skewedPartitions.nonEmpty) { val optimizedSmj = smj.transformDown { case sort @ SortExec(_, _, shuffleStage: ShuffleQueryStageExec, _) => - val shuffleStage = sort.child.asInstanceOf[ShuffleQueryStageExec] val newStage = shuffleStage.copy( excludedPartitions = skewedPartitions.toSet) newStage.resultOption = shuffleStage.resultOption @@ -219,20 +218,20 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { /** * A wrapper of shuffle query stage, which submits one reduce task to read a single - * shuffle partition 'partitionIndex' produced by the mappers in range [startMapId, endMapId). + * shuffle partition 'partitionIndex' produced by the mappers in range [startMapIndex, endMapIndex). * This is used to handle the skewed partitions. * * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange * node during canonicalization. * @param partitionIndex The pre shuffle partition index. - * @param startMapId The start map id. - * @param endMapId The end map id. + * @param startMapIndex The start map index. + * @param endMapIndex The end map index. */ case class SkewedShufflePartitionReader( child: QueryStageExec, partitionIndex: Int, - startMapId: Int, - endMapId: Int) extends LeafExecNode { + startMapIndex: Int, + endMapIndex: Int) extends LeafExecNode { override def output: Seq[Attribute] = child.output @@ -245,7 +244,7 @@ case class SkewedShufflePartitionReader( if (cachedSkewedShuffleRDD == null) { cachedSkewedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.shuffle.createSkewedShuffleRDD(partitionIndex, startMapId, endMapId) + stage.shuffle.createSkewedShuffleRDD(partitionIndex, startMapIndex, endMapIndex) case _ => throw new IllegalStateException("operating on canonicalization plan") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 349f2037b773..cc3fbde0ec67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -166,15 +166,7 @@ case class ShuffleQueryStageExec( private def getPartitionIndexRanges(): Array[(Int, Int)] = { val length = shuffle.shuffleDependency.partitioner.numPartitions - val partitionStartIndices = ArrayBuffer[Int]() - val partitionEndIndices = ArrayBuffer[Int]() - (0 until length).map { i => - if (!excludedPartitions.contains(i)) { - partitionStartIndices += i - partitionEndIndices += i + 1 - } - } - partitionStartIndices.zip(partitionEndIndices).toArray + (0 until length).filterNot(excludedPartitions.contains).map(i => (i, i + 1)).toArray } private var cachedShuffleRDD: RDD[InternalRow] = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala index 2dbe6614251b..52f793b24aa1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala @@ -33,25 +33,25 @@ class SkewedShuffledRowRDDPartition(override val index: Int) extends Partition * partitions from mappers. * * This RDD takes a [[ShuffleDependency]] (`dependency`), a partitionIndex - * and the range of startMapId to endMapId. - * + * and the range of startMapIndex to endMapIndex. */ class SkewedShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], partitionIndex: Int, - startMapId: Int, - endMapId: Int, + startMapIndex: Int, + endMapIndex: Int, metrics: Map[String, SQLMetric]) extends RDD[InternalRow](dependency.rdd.context, Nil) { override def getDependencies: Seq[Dependency[_]] = List(dependency) + override def getPartitions: Array[Partition] = { Array(new SkewedShuffledRowRDDPartition(0)) } override def getPreferredLocations(partition: Partition): Seq[String] = { val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - tracker.getMapLocation(dependency, startMapId, endMapId) + tracker.getMapLocation(dependency, startMapIndex, endMapIndex) } override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { @@ -62,8 +62,8 @@ class SkewedShuffledRowRDD( val reader = SparkEnv.get.shuffleManager.getReaderForRange( dependency.shuffleHandle, - startMapId, - endMapId, + startMapIndex, + endMapIndex, partitionIndex, partitionIndex + 1, context, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 9befd00b0918..ffcd6c778335 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -102,9 +102,10 @@ case class ShuffleExchangeExec( def createSkewedShuffleRDD( partitionIndex: Int, - startMapId: Int, - endMapId: Int): SkewedShuffledRowRDD = { - new SkewedShuffledRowRDD(shuffleDependency, partitionIndex, startMapId, endMapId, readMetrics) + startMapIndex: Int, + endMapIndex: Int): SkewedShuffledRowRDD = { + new SkewedShuffledRowRDD(shuffleDependency, + partitionIndex, startMapIndex, endMapIndex, readMetrics) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 4dc735b58f24..ae0ba1986474 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -579,17 +579,7 @@ class AdaptiveQueryExecSuite } } - test("SPARK-29544: adaptive skew join both in left and right for inner join," + - " left outer join and right outer join") { - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 2 as key1", "id as value1") - .createOrReplaceTempView("skewData1") - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 1 as key2", "id as value2") - .createOrReplaceTempView("skewData2") - + test("SPARK-29544: adaptive skew join with different join types") { Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", @@ -597,135 +587,143 @@ class AdaptiveQueryExecSuite SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> reducePostShufflePartitionsEnabled, SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "700") { - // skewed inner join optimization - val (innerPlan, innerAdaptivePlan) = runAdaptiveAndVerifyResult( - "SELECT * FROM skewData1 join skewData2 ON key1 = key2") - val innerSmj = findTopLevelSortMergeJoin(innerPlan) - assert(innerSmj.size == 1) - // left stats: [3496, 0, 0, 0, 4014] - // right stats:[6292, 0, 0, 0, 0] - // the partition 0 in both left and right side are all skewed. - // And divide into 5 splits both in left and right (the max splits number). - // So there are 5 x 5 smjs for partition 0. - // Partition 4 in left side is skewed and is divided into 5 splits. - // The right side of partition 4 is not skewed. - // So there are 5 smjs for partition 4. - // So total (25 + 5 + 1) smjs. - // Union - // +- SortMergeJoin - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // . - // . - // . - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - - val innerSmjAfter = findTopLevelSortMergeJoin(innerAdaptivePlan) - assert(innerSmjAfter.size == 31) - - // skewed left outer join optimization - val (leftPlan, leftAdaptivePlan) = runAdaptiveAndVerifyResult( - "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") - val leftSmj = findTopLevelSortMergeJoin(leftPlan) - assert(leftSmj.size == 1) - // left stats: [3496, 0, 0, 0, 4014] - // right stats:[6292, 0, 0, 0, 0] - // The partition 0 in both left and right are all skewed. - // The partition 4 in left side is skewed. - // But for left outer join, we don't split the right partition even skewed. - // So the partition 0 in left side is divided into 5 splits(the max split number). - // the partition 4 in left side is divided into 5 splits(the max split number). - // So total (5 + 5 + 1) smjs. - // Union - // +- SortMergeJoin - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // . - // . - // . - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - - val leftSmjAfter = findTopLevelSortMergeJoin(leftAdaptivePlan) - assert(leftSmjAfter.size == 11) - - // skewed right outer join optimization - val (rightPlan, rightAdaptivePlan) = runAdaptiveAndVerifyResult( - "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") - val rightSmj = findTopLevelSortMergeJoin(rightPlan) - assert(rightSmj.size == 1) - // left stats: [3496, 0, 0, 0, 4014] - // right stats:[6292, 0, 0, 0, 0] - // The partition 0 in both left and right side are all skewed. - // And the partition 4 in left side is skewed. - // But for right outer join, we don't split the left partition even skewed. - // And divide right side into 5 splits(the max split number) - // So total 6 smjs. - // Union - // +- SortMergeJoin - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // . - // . - // . - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - - val rightSmjAfter = findTopLevelSortMergeJoin(rightAdaptivePlan) - assert(rightSmjAfter.size == 6) + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 2 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + // skewed inner join optimization + val (innerPlan, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 join skewData2 ON key1 = key2") + val innerSmj = findTopLevelSortMergeJoin(innerPlan) + assert(innerSmj.size == 1) + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // the partition 0 in both left and right side are all skewed. + // And divide into 5 splits both in left and right (the max splits number). + // So there are 5 x 5 smjs for partition 0. + // Partition 4 in left side is skewed and is divided into 5 splits. + // The right side of partition 4 is not skewed. + // So there are 5 smjs for partition 4. + // So total (25 + 5 + 1) smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val innerSmjAfter = findTopLevelSortMergeJoin(innerAdaptivePlan) + assert(innerSmjAfter.size == 31) + + // skewed left outer join optimization + val (leftPlan, leftAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") + val leftSmj = findTopLevelSortMergeJoin(leftPlan) + assert(leftSmj.size == 1) + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // The partition 0 in both left and right are all skewed. + // The partition 4 in left side is skewed. + // But for left outer join, we don't split the right partition even skewed. + // So the partition 0 in left side is divided into 5 splits(the max split number). + // the partition 4 in left side is divided into 5 splits(the max split number). + // So total (5 + 5 + 1) smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val leftSmjAfter = findTopLevelSortMergeJoin(leftAdaptivePlan) + assert(leftSmjAfter.size == 11) + + // skewed right outer join optimization + val (rightPlan, rightAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") + val rightSmj = findTopLevelSortMergeJoin(rightPlan) + assert(rightSmj.size == 1) + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // The partition 0 in both left and right side are all skewed. + // And the partition 4 in left side is skewed. + // But for right outer join, we don't split the left partition even skewed. + // And divide right side into 5 splits(the max split number) + // So total 6 smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val rightSmjAfter = findTopLevelSortMergeJoin(rightAdaptivePlan) + assert(rightSmjAfter.size == 6) + } } } - spark.catalog.dropTempView("skewData1") - spark.catalog.dropTempView("skewData2") } test("SPARK-30291: AQE should catch the exceptions when doing materialize") { From a22b804a8ff12502a672d084e9a3c4e7292f73e4 Mon Sep 17 00:00:00 2001 From: jiake Date: Fri, 10 Jan 2020 12:18:19 +0800 Subject: [PATCH 21/23] resolve comments --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 5506a4836e67..6cb44a316059 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -753,11 +753,11 @@ private[spark] class MapOutputTrackerMaster( } override def getMapSizesByRange( - shuffleId: Int, - startMapIndex: Int, - endMapIndex: Int, - startPartition: Int, - endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + shuffleId: Int, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, mappers $startMapIndex - $endMapIndex" + s"partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { From cee1c8cb7b4c4714dd3d59fbaa693d1dab217767 Mon Sep 17 00:00:00 2001 From: jiake Date: Fri, 10 Jan 2020 21:03:59 +0800 Subject: [PATCH 22/23] resolve the comments --- .../scala/org/apache/spark/MapOutputTracker.scala | 4 +++- .../adaptive/OptimizeSkewedPartitions.scala | 12 +++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 6cb44a316059..f229061a6d0f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -758,7 +758,7 @@ private[spark] class MapOutputTrackerMaster( endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, mappers $startMapIndex - $endMapIndex" + + logDebug(s"Fetching outputs for shuffle $shuffleId, mappers $startMapIndex-$endMapIndex" + s"partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => @@ -822,6 +822,8 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + logDebug(s"Fetching outputs for shuffle $shuffleId, mappers $startMapIndex-$endMapIndex" + + s"partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId, conf) try { MapOutputTracker.convertMapStatuses( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala index 5b2e5e0a9701..1bbab7e9b42c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala @@ -74,7 +74,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS), mapPartitionSizes.length) val avgPartitionSize = mapPartitionSizes.sum / maxSplits val advisoryPartitionSize = math.max(avgPartitionSize, - conf.getConf(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE)) + conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD)) val partitionIndices = mapPartitionSizes.indices val partitionStartIndices = ArrayBuffer[Int]() var postMapPartitionSize = mapPartitionSizes(0) @@ -168,11 +168,13 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { // obtaining the raw data size of per partition, val leftSkewedReader = SkewedShufflePartitionReader( left, partitionId, leftMapIdStartIndices(i), leftEndMapId) + val leftSort = smj.left.asInstanceOf[SortExec].copy(child = leftSkewedReader) val rightSkewedReader = SkewedShufflePartitionReader(right, partitionId, rightMapIdStartIndices(j), rightEndMapId) - subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, - leftSkewedReader, rightSkewedReader) + val rightSort = smj.right.asInstanceOf[SortExec].copy(child = rightSkewedReader) + subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, + leftSort, rightSort) } } } @@ -240,6 +242,10 @@ case class SkewedShufflePartitionReader( } private var cachedSkewedShuffleRDD: SkewedShuffledRowRDD = null + override def nodeName: String = s"SkewedShuffleReader SkewedShuffleQueryStage: ${child}" + + s" SkewedPartition: ${partitionIndex} startMapIndex: ${startMapIndex}" + + s" endMapIndex: ${endMapIndex}" + override def doExecute(): RDD[InternalRow] = { if (cachedSkewedShuffleRDD == null) { cachedSkewedShuffleRDD = child match { From 4abad37ef2e4a64be9b01193444f2e04f9d641fa Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 14 Jan 2020 16:09:24 +0800 Subject: [PATCH 23/23] create partial shuffle reader --- .../adaptive/AdaptiveSparkPlanExec.scala | 2 +- .../adaptive/OptimizeLocalShuffleReader.scala | 4 +- ...titions.scala => OptimizeSkewedJoin.scala} | 75 +++++++++++++------ .../execution/adaptive/QueryStageExec.scala | 24 +----- .../adaptive/ReduceNumShufflePartitions.scala | 45 +++++++---- .../ReduceNumShufflePartitionsSuite.scala | 4 +- .../adaptive/AdaptiveQueryExecSuite.scala | 2 +- 7 files changed, 93 insertions(+), 63 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/{OptimizeSkewedPartitions.scala => OptimizeSkewedJoin.scala} (81%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 42e18f2708c9..65d0f6a6a41d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -90,7 +90,7 @@ case class AdaptiveSparkPlanExec( // Here the 'OptimizeSkewedPartitions' rule should be executed // before 'ReduceNumShufflePartitions', as the skewed partition handled // in 'OptimizeSkewedPartitions' rule, should be omitted in 'ReduceNumShufflePartitions'. - OptimizeSkewedPartitions(conf), + OptimizeSkewedJoin(conf), ReduceNumShufflePartitions(conf), // The rule of 'OptimizeLocalShuffleReader' need to make use of the 'partitionStartIndices' // in 'ReduceNumShufflePartitions' rule. So it must be after 'ReduceNumShufflePartitions' rule. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 72d74818e8a5..fd55f6c5eb4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -165,9 +165,9 @@ case class LocalShuffleReaderExec( // before shuffle. if (partitionStartIndicesPerMapper.forall(_.length == 1)) { child match { - case ShuffleQueryStageExec(_, s: ShuffleExchangeExec, _) => + case ShuffleQueryStageExec(_, s: ShuffleExchangeExec) => s.child.outputPartitioning - case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeExec), _) => + case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeExec)) => s.child.outputPartitioning match { case e: Expression => r.updateAttr(e).asInstanceOf[Partitioning] case other => other diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala similarity index 81% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala index 1bbab7e9b42c..75d4184a2c14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedPartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -28,10 +28,11 @@ 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.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.internal.SQLConf -case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { +case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { private val supportedJoinTypes = Inner :: Cross :: LeftSemi :: LeftAnti :: LeftOuter :: RightOuter :: Nil @@ -115,8 +116,8 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { def handleSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { case smj @ SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, - SortExec(_, _, left: ShuffleQueryStageExec, _), - SortExec(_, _, right: ShuffleQueryStageExec, _)) + s1 @ SortExec(_, _, left: ShuffleQueryStageExec, _), + s2 @ SortExec(_, _, right: ShuffleQueryStageExec, _)) if supportedJoinTypes.contains(joinType) => val leftStats = getStatistics(left) val rightStats = getStatistics(right) @@ -166,15 +167,12 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { } // TODO: we may can optimize the sort merge join to broad cast join after // obtaining the raw data size of per partition, - val leftSkewedReader = SkewedShufflePartitionReader( + val leftSkewedReader = SkewedPartitionReaderExec( left, partitionId, leftMapIdStartIndices(i), leftEndMapId) - val leftSort = smj.left.asInstanceOf[SortExec].copy(child = leftSkewedReader) - - val rightSkewedReader = SkewedShufflePartitionReader(right, partitionId, - rightMapIdStartIndices(j), rightEndMapId) - val rightSort = smj.right.asInstanceOf[SortExec].copy(child = rightSkewedReader) - subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, - leftSort, rightSort) + val rightSkewedReader = SkewedPartitionReaderExec(right, partitionId, + rightMapIdStartIndices(j), rightEndMapId) + subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, + s1.copy(child = leftSkewedReader), s2.copy(child = rightSkewedReader)) } } } @@ -182,10 +180,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { if (skewedPartitions.nonEmpty) { val optimizedSmj = smj.transformDown { case sort @ SortExec(_, _, shuffleStage: ShuffleQueryStageExec, _) => - val newStage = shuffleStage.copy( - excludedPartitions = skewedPartitions.toSet) - newStage.resultOption = shuffleStage.resultOption - sort.copy(child = newStage) + sort.copy(child = PartialShuffleReaderExec(shuffleStage, skewedPartitions.toSet)) } subJoins += optimizedSmj UnionExec(subJoins) @@ -221,7 +216,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { /** * A wrapper of shuffle query stage, which submits one reduce task to read a single * shuffle partition 'partitionIndex' produced by the mappers in range [startMapIndex, endMapIndex). - * This is used to handle the skewed partitions. + * This is used to increase the parallelism when reading skewed partitions. * * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange * node during canonicalization. @@ -229,7 +224,7 @@ case class OptimizeSkewedPartitions(conf: SQLConf) extends Rule[SparkPlan] { * @param startMapIndex The start map index. * @param endMapIndex The end map index. */ -case class SkewedShufflePartitionReader( +case class SkewedPartitionReaderExec( child: QueryStageExec, partitionIndex: Int, startMapIndex: Int, @@ -242,10 +237,6 @@ case class SkewedShufflePartitionReader( } private var cachedSkewedShuffleRDD: SkewedShuffledRowRDD = null - override def nodeName: String = s"SkewedShuffleReader SkewedShuffleQueryStage: ${child}" + - s" SkewedPartition: ${partitionIndex} startMapIndex: ${startMapIndex}" + - s" endMapIndex: ${endMapIndex}" - override def doExecute(): RDD[InternalRow] = { if (cachedSkewedShuffleRDD == null) { cachedSkewedShuffleRDD = child match { @@ -258,3 +249,45 @@ case class SkewedShufflePartitionReader( cachedSkewedShuffleRDD } } + +/** + * A wrapper of shuffle query stage, which skips some partitions when reading the shuffle blocks. + * + * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange node during + * canonicalization. + * @param excludedPartitions The partitions to skip when reading. + */ +case class PartialShuffleReaderExec( + child: QueryStageExec, + excludedPartitions: Set[Int]) extends UnaryExecNode { + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = { + UnknownPartitioning(1) + } + + private def shuffleExchange(): ShuffleExchangeExec = child match { + case stage: ShuffleQueryStageExec => stage.shuffle + case _ => + throw new IllegalStateException("operating on canonicalization plan") + } + + private def getPartitionIndexRanges(): Array[(Int, Int)] = { + val length = shuffleExchange().shuffleDependency.partitioner.numPartitions + (0 until length).filterNot(excludedPartitions.contains).map(i => (i, i + 1)).toArray + } + + private var cachedShuffleRDD: RDD[InternalRow] = null + + override def doExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + cachedShuffleRDD = if (excludedPartitions.isEmpty) { + child.execute() + } else { + shuffleExchange().createShuffledRDD(Some(getPartitionIndexRanges())) + } + } + cachedShuffleRDD + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index cc3fbde0ec67..d5dc1be63f06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.adaptive -import scala.collection.mutable.ArrayBuffer import scala.concurrent.Future import org.apache.spark.{FutureAction, MapOutputStatistics} @@ -135,8 +134,7 @@ abstract class QueryStageExec extends LeafExecNode { */ case class ShuffleQueryStageExec( override val id: Int, - override val plan: SparkPlan, - val excludedPartitions: Set[Int] = Set.empty) extends QueryStageExec { + override val plan: SparkPlan) extends QueryStageExec { @transient val shuffle = plan match { case s: ShuffleExchangeExec => s @@ -163,26 +161,6 @@ case class ShuffleQueryStageExec( case _ => } } - - private def getPartitionIndexRanges(): Array[(Int, Int)] = { - val length = shuffle.shuffleDependency.partitioner.numPartitions - (0 until length).filterNot(excludedPartitions.contains).map(i => (i, i + 1)).toArray - } - - private var cachedShuffleRDD: RDD[InternalRow] = null - - override def doExecute(): RDD[InternalRow] = { - if (cachedShuffleRDD == null) { - cachedShuffleRDD = excludedPartitions match { - case e if e.isEmpty => - plan.execute() - case _ => - shuffle.createShuffledRDD( - Some(getPartitionIndexRanges())) - } - } - cachedShuffleRDD - } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index ed8769c99c32..2c50b638b4d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.adaptive -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.MapOutputStatistics import org.apache.spark.rdd.RDD @@ -54,8 +54,9 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { if (!conf.reducePostShufflePartitionsEnabled) { return plan } - // we need skip the leaf node of 'SkewedShufflePartitionReader' - val leafNodes = plan.collectLeaves().filter(!_.isInstanceOf[SkewedShufflePartitionReader]) + // 'SkewedShufflePartitionReader' is added by us, so it's safe to ignore it when changing + // number of reducers. + val leafNodes = plan.collectLeaves().filter(!_.isInstanceOf[SkewedPartitionReaderExec]) if (!leafNodes.forall(_.isInstanceOf[QueryStageExec])) { // If not all leaf nodes are query stages, it's not safe to reduce the number of // shuffle partitions, because we may break the assumption that all children of a spark plan @@ -63,13 +64,18 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { return plan } - def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { + def collectShuffles(plan: SparkPlan): Seq[SparkPlan] = plan match { case _: LocalShuffleReaderExec => Nil + case p: PartialShuffleReaderExec => Seq(p) case stage: ShuffleQueryStageExec => Seq(stage) - case _ => plan.children.flatMap(collectShuffleStages) + case _ => plan.children.flatMap(collectShuffles) } - val shuffleStages = collectShuffleStages(plan) + val shuffles = collectShuffles(plan) + val shuffleStages = shuffles.map { + case PartialShuffleReaderExec(s: ShuffleQueryStageExec, _) => s + case s: ShuffleQueryStageExec => s + } // ShuffleExchanges introduced by repartition do not support changing the number of partitions. // We change the number of partitions in the stage only if all the ShuffleExchanges support it. if (!shuffleStages.forall(_.shuffle.canChangeNumPartitions)) { @@ -88,18 +94,31 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // partition) and a result of a SortMergeJoin (multiple partitions). val distinctNumPreShufflePartitions = validMetrics.map(stats => stats.bytesByPartitionId.length).distinct - val distinctExcludedPartitions = shuffleStages.map(_.excludedPartitions).distinct + val distinctExcludedPartitions = shuffles.map { + case PartialShuffleReaderExec(_, excludedPartitions) => excludedPartitions + case _: ShuffleQueryStageExec => Set.empty[Int] + }.distinct if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1 && distinctExcludedPartitions.length == 1) { - val excludedPartitions = shuffleStages.head.excludedPartitions + val excludedPartitions = distinctExcludedPartitions.head val partitionIndices = estimatePartitionStartAndEndIndices( validMetrics.toArray, excludedPartitions) // This transformation adds new nodes, so we must use `transformUp` here. - plan.transformUp { - // even for shuffle exchange whose input RDD has 0 partition, we should still update its - // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same - // number of output partitions. - case stage: ShuffleQueryStageExec => + // Even for shuffle exchange whose input RDD has 0 partition, we should still update its + // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same + // number of output partitions. + val visitedStages = HashSet.empty[Int] + plan.transformDown { + // Replace `PartialShuffleReaderExec` with `CoalescedShuffleReaderExec`, which keeps the + // "excludedPartition" requirement and also merges some partitions. + case PartialShuffleReaderExec(stage: ShuffleQueryStageExec, _) => + visitedStages.add(stage.id) + CoalescedShuffleReaderExec(stage, partitionIndices) + + // We are doing `transformDown`, so the `ShuffleQueryStageExec` may already be optimized + // and wrapped by `CoalescedShuffleReaderExec`. + case stage: ShuffleQueryStageExec if !visitedStages.contains(stage.id) => + visitedStages.add(stage.id) CoalescedShuffleReaderExec(stage, partitionIndices) } } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 463b023cd3c8..04b4d4f29f85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -533,7 +533,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val finalPlan = resultDf.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan assert(finalPlan.collect { - case ShuffleQueryStageExec(_, r: ReusedExchangeExec, _) => r + case ShuffleQueryStageExec(_, r: ReusedExchangeExec) => r }.length == 2) assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 3) @@ -566,7 +566,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val reusedStages = level1Stages.flatMap { stage => stage.plan.collect { - case ShuffleQueryStageExec(_, r: ReusedExchangeExec, _) => r + case ShuffleQueryStageExec(_, r: ReusedExchangeExec) => r } } assert(reusedStages.length == 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index ae0ba1986474..c2daae071afc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -92,7 +92,7 @@ class AdaptiveQueryExecSuite private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = { collectInPlanAndSubqueries(plan) { - case ShuffleQueryStageExec(_, e: ReusedExchangeExec, _) => e + case ShuffleQueryStageExec(_, e: ReusedExchangeExec) => e case BroadcastQueryStageExec(_, e: ReusedExchangeExec) => e } }