From 7f0c2c96126cc11704ca720f769baf11b7dce25c Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 22 Mar 2018 09:17:01 +0800 Subject: [PATCH 01/21] Add QueryStage and the framework for adaptive execution --- .../apache/spark/MapOutputStatistics.scala | 1 + .../apache/spark/sql/internal/SQLConf.scala | 24 +- .../spark/sql/execution/QueryExecution.scala | 17 +- .../spark/sql/execution/SparkPlanInfo.scala | 2 + .../execution/adaptive/PlanQueryStage.scala | 79 +++++++ .../sql/execution/adaptive/QueryStage.scala | 222 ++++++++++++++++++ .../execution/adaptive/QueryStageInput.scala | 105 +++++++++ .../exchange/EnsureRequirements.scala | 120 +--------- .../exchange/ExchangeCoordinator.scala | 182 ++------------ .../exchange/ShuffleExchangeExec.scala | 58 ++--- .../execution/ui/SQLAppStatusListener.scala | 54 +++-- .../spark/sql/execution/ui/SQLListener.scala | 7 + .../sql/execution/ui/SparkPlanGraph.scala | 12 + .../org/apache/spark/sql/DatasetSuite.scala | 2 +- .../execution/ExchangeCoordinatorSuite.scala | 100 ++++---- .../spark/sql/execution/PlannerSuite.scala | 9 +- 16 files changed, 600 insertions(+), 394 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala diff --git a/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala b/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala index f8a6f1d0d8cbb..c02e48c9815fa 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala @@ -25,3 +25,4 @@ package org.apache.spark * (may be inexact due to use of compressed map statuses) */ private[spark] class MapOutputStatistics(val shuffleId: Int, val bytesByPartitionId: Array[Long]) + extends Serializable 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 9804af7dff179..214c36e08b269 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 @@ -282,14 +282,19 @@ object SQLConf { val SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS = buildConf("spark.sql.adaptive.minNumPostShufflePartitions") - .internal() - .doc("The advisory minimal number of post-shuffle partitions provided to " + - "ExchangeCoordinator. This setting is used in our test to make sure we " + - "have enough parallelism to expose issues that will not be exposed with a " + - "single partition. When the value is a non-positive value, this setting will " + - "not be provided to ExchangeCoordinator.") + .doc("The advisory minimum number of post-shuffle partitions used in adaptive execution.") + .intConf + .checkValue(numPartitions => numPartitions > 0, "The minimum shuffle partition number " + + "must be a positive integer.") + .createWithDefault(1) + + val SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS = + buildConf("spark.sql.adaptive.maxNumPostShufflePartitions") + .doc("The advisory maximum number of post-shuffle partitions used in adaptive execution.") .intConf - .createWithDefault(-1) + .checkValue(numPartitions => numPartitions > 0, "The maximum shuffle partition number " + + "must be a positive integer.") + .createWithDefault(500) val SUBEXPRESSION_ELIMINATION_ENABLED = buildConf("spark.sql.subexpressionElimination.enabled") @@ -1735,8 +1740,9 @@ class SQLConf extends Serializable with Logging { def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) - def minNumPostShufflePartitions: Int = - getConf(SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS) + def minNumPostShufflePartitions: Int = getConf(SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS) + + def maxNumPostShufflePartitions: Int = getConf(SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS) def minBatchesToRetain: Int = getConf(MIN_BATCHES_TO_RETAIN) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 72499aa936a56..b5e7c499c6ed1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.execution.adaptive.PlanQueryStage import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils @@ -93,7 +94,11 @@ class QueryExecution( * row format conversions as needed. */ protected def prepareForExecution(plan: SparkPlan): SparkPlan = { - preparations.foldLeft(plan) { case (sp, rule) => rule.apply(sp) } + if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { + adaptivePreparations.foldLeft(plan) { case (sp, rule) => rule.apply(sp)} + } else { + preparations.foldLeft(plan) { case (sp, rule) => rule.apply(sp)} + } } /** A sequence of rules that will be applied in order to the physical plan before execution. */ @@ -104,6 +109,16 @@ class QueryExecution( ReuseExchange(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf)) + protected def adaptivePreparations: Seq[Rule[SparkPlan]] = Seq( + python.ExtractPythonUDFs, + PlanSubqueries(sparkSession), + EnsureRequirements(sparkSession.sessionState.conf), + ReuseSubquery(sparkSession.sessionState.conf), + // PlanQueryStage needs to be the last rule because it divides the plan into multiple sub-trees + // by inserting leaf node QueryStageInput. Transforming the plan after applying this rule will + // only transform node in a sub-tree. + PlanQueryStage(sparkSession.sessionState.conf)) + def simpleString: String = withRedaction { val concat = new StringConcat() concat.append("== Physical Plan ==\n") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index f554ff0aa775f..ca46b1e940e46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.execution.adaptive.QueryStageInput import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo import org.apache.spark.sql.internal.SQLConf @@ -52,6 +53,7 @@ private[execution] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val children = plan match { case ReusedExchangeExec(_, child) => child :: Nil + case i: QueryStageInput => i.childStage :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala new file mode 100644 index 0000000000000..ab2b6e9dfdec1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.adaptive + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.command.ExecutedCommandExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ShuffleExchangeExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +/** + * Divide the spark plan into multiple QueryStages. For each Exchange in the plan, it adds a + * QueryStage and a QueryStageInput. If reusing Exchange is enabled, it finds duplicated exchanges + * and uses the same QueryStage for all the references. + */ +case class PlanQueryStage(conf: SQLConf) extends Rule[SparkPlan] { + + def apply(plan: SparkPlan): SparkPlan = { + + val newPlan = if (!conf.exchangeReuseEnabled) { + plan.transformUp { + case e: ShuffleExchangeExec => + ShuffleQueryStageInput(ShuffleQueryStage(e), e.output) + case e: BroadcastExchangeExec => + BroadcastQueryStageInput(BroadcastQueryStage(e), e.output) + } + } else { + // Build a hash map using schema of exchanges to avoid O(N*N) sameResult calls. + val stages = mutable.HashMap[StructType, ArrayBuffer[QueryStage]]() + + plan.transformUp { + case exchange: Exchange => + val sameSchema = stages.getOrElseUpdate(exchange.schema, ArrayBuffer[QueryStage]()) + val samePlan = sameSchema.find { s => + exchange.sameResult(s.child) + } + if (samePlan.isDefined) { + // Keep the output of this exchange, the following plans require that to resolve + // attributes. + exchange match { + case e: ShuffleExchangeExec => ShuffleQueryStageInput( + samePlan.get.asInstanceOf[ShuffleQueryStage], exchange.output) + case e: BroadcastExchangeExec => BroadcastQueryStageInput( + samePlan.get.asInstanceOf[BroadcastQueryStage], exchange.output) + } + } else { + val queryStageInput = exchange match { + case e: ShuffleExchangeExec => + ShuffleQueryStageInput(ShuffleQueryStage(e), e.output) + case e: BroadcastExchangeExec => + BroadcastQueryStageInput(BroadcastQueryStage(e), e.output) + } + sameSchema += queryStageInput.childStage + queryStageInput + } + } + } + ResultQueryStage(newPlan) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala new file mode 100644 index 0000000000000..83d34681c7c4f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala @@ -0,0 +1,222 @@ +/* + * 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.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration + +import org.apache.spark.MapOutputStatistics +import org.apache.spark.broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.exchange._ +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate +import org.apache.spark.util.ThreadUtils + +/** + * In adaptive execution mode, an execution plan is divided into multiple QueryStages. Each + * QueryStage is a sub-tree that runs in a single stage. + */ +abstract class QueryStage extends UnaryExecNode { + + var child: SparkPlan + + // Ignore this wrapper for canonicalizing. + override def doCanonicalize(): SparkPlan = child.canonicalized + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + /** + * Execute childStages and wait until all stages are completed. Use a thread pool to avoid + * blocking on one child stage. + */ + def executeChildStages(): Unit = { + // Handle broadcast stages + val broadcastQueryStages: Seq[BroadcastQueryStage] = child.collect { + case bqs: BroadcastQueryStageInput => bqs.childStage + } + val broadcastFutures = broadcastQueryStages.map { queryStage => + Future { queryStage.prepareBroadcast() }(QueryStage.executionContext) + } + + // Submit shuffle stages + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + val shuffleQueryStages: Seq[ShuffleQueryStage] = child.collect { + case sqs: ShuffleQueryStageInput => sqs.childStage + } + val shuffleStageFutures = shuffleQueryStages.map { queryStage => + Future { + SQLExecution.withExecutionId(sqlContext.sparkContext, executionId) { + queryStage.execute() + } + }(QueryStage.executionContext) + } + + ThreadUtils.awaitResult( + Future.sequence(broadcastFutures)(implicitly, QueryStage.executionContext), Duration.Inf) + ThreadUtils.awaitResult( + Future.sequence(shuffleStageFutures)(implicitly, QueryStage.executionContext), Duration.Inf) + } + + /** + * Before executing the plan in this query stage, we execute all child stages, optimize the plan + * in this stage and determine the reducer number based on the child stages' statistics. Finally + * we do a codegen for this query stage and update the UI with the new plan. + */ + def prepareExecuteStage(): Unit = { + // 1. Execute childStages + executeChildStages() + // It is possible to optimize this stage's plan here based on the child stages' statistics. + + // 2. Determine reducer number + val queryStageInputs: Seq[ShuffleQueryStageInput] = child.collect { + case input: ShuffleQueryStageInput => input + } + val childMapOutputStatistics = queryStageInputs.map(_.childStage.mapOutputStatistics) + .filter(_ != null).toArray + if (childMapOutputStatistics.length > 0) { + val exchangeCoordinator = new ExchangeCoordinator( + conf.targetPostShuffleInputSize, + conf.minNumPostShufflePartitions) + + val partitionStartIndices = + exchangeCoordinator.estimatePartitionStartIndices(childMapOutputStatistics) + child = child.transform { + case ShuffleQueryStageInput(childStage, output, _) => + ShuffleQueryStageInput(childStage, output, Some(partitionStartIndices)) + } + } + + // 3. Codegen and update the UI + child = CollapseCodegenStages(sqlContext.conf).apply(child) + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + if (executionId != null && executionId.nonEmpty) { + val queryExecution = SQLExecution.getQueryExecution(executionId.toLong) + sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( + executionId.toLong, + queryExecution.toString, + SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan))) + } + } + + // Caches the created ShuffleRowRDD so we can reuse that. + private var cachedRDD: RDD[InternalRow] = null + + def executeStage(): RDD[InternalRow] = child.execute() + + /** + * A QueryStage can be reused like Exchange. It is possible that multiple threads try to submit + * the same QueryStage. Use synchronized to make sure it is executed only once. + */ + override def doExecute(): RDD[InternalRow] = synchronized { + if (cachedRDD == null) { + prepareExecuteStage() + cachedRDD = executeStage() + } + cachedRDD + } + + override def executeCollect(): Array[InternalRow] = { + prepareExecuteStage() + child.executeCollect() + } + + override def executeToIterator(): Iterator[InternalRow] = { + prepareExecuteStage() + child.executeToIterator() + } + + override def executeTake(n: Int): Array[InternalRow] = { + prepareExecuteStage() + child.executeTake(n) + } + + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + builder: StringBuilder, + verbose: Boolean, + prefix: String = "", + addSuffix: Boolean = false): StringBuilder = { + child.generateTreeString(depth, lastChildren, builder, verbose, "*") + } +} + +/** + * The last QueryStage of an execution plan. + */ +case class ResultQueryStage(var child: SparkPlan) extends QueryStage + +/** + * A shuffle QueryStage whose child is a ShuffleExchange. + */ +case class ShuffleQueryStage(var child: SparkPlan) extends QueryStage { + + protected var _mapOutputStatistics: MapOutputStatistics = null + + def mapOutputStatistics: MapOutputStatistics = _mapOutputStatistics + + override def executeStage(): RDD[InternalRow] = { + child match { + case e: ShuffleExchangeExec => + val result = e.eagerExecute() + _mapOutputStatistics = e.mapOutputStatistics + result + case _ => throw new IllegalArgumentException( + "The child of ShuffleQueryStage must be a ShuffleExchange.") + } + } +} + +/** + * A broadcast QueryStage whose child is a BroadcastExchangeExec. + */ +case class BroadcastQueryStage(var child: SparkPlan) extends QueryStage { + override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + child.executeBroadcast() + } + + private var prepared = false + + def prepareBroadcast() : Unit = synchronized { + if (!prepared) { + executeChildStages() + child = CollapseCodegenStages(sqlContext.conf).apply(child) + // After child stages are completed, prepare() triggers the broadcast. + prepare() + prepared = true + } + } + + override def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException( + "BroadcastExchange does not support the execute() code path.") + } +} + +object QueryStage { + private[execution] val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("adaptive-query-stage")) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala new file mode 100644 index 0000000000000..887f815b6117a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala @@ -0,0 +1,105 @@ +/* + * 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.broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} +import org.apache.spark.sql.execution._ + +/** + * QueryStageInput is the leaf node of a QueryStage and is used to hide its child stage. It gets + * the result of its child stage and serves it as the input of the QueryStage. A QueryStage knows + * its child stages by collecting all the QueryStageInputs. + */ +abstract class QueryStageInput extends LeafExecNode { + + def childStage: QueryStage + + // Ignore this wrapper for canonicalizing. + override def doCanonicalize(): SparkPlan = childStage.canonicalized + + // Similar to ReusedExchangeExec, two QueryStageInputs can reference to the same childStage. + // QueryStageInput can have distinct set of output attribute ids from its childStage, we need + // to update the attribute ids in outputPartitioning and outputOrdering. + private lazy val updateAttr: Expression => Expression = { + val originalAttrToNewAttr = AttributeMap(childStage.output.zip(output)) + e => e.transform { + case attr: Attribute => originalAttrToNewAttr.getOrElse(attr, attr) + } + } + + override def outputPartitioning: Partitioning = childStage.outputPartitioning match { + case h: HashPartitioning => h.copy(expressions = h.expressions.map(updateAttr)) + case other => other + } + + override def outputOrdering: Seq[SortOrder] = { + childStage.outputOrdering.map(updateAttr(_).asInstanceOf[SortOrder]) + } + + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + builder: StringBuilder, + verbose: Boolean, + prefix: String = "", + addSuffix: Boolean = false): StringBuilder = { + childStage.generateTreeString(depth, lastChildren, builder, verbose, "*") + } +} + +/** + * A QueryStageInput whose child stage is a ShuffleQueryStage. It returns a new ShuffledRowRDD + * based on the the child stage's result RDD and the specified partitionStartIndices. If the + * child stage is reused by another ShuffleQueryStageInput, they can return RDDs with different + * partitionStartIndices. + */ +case class ShuffleQueryStageInput( + childStage: ShuffleQueryStage, + override val output: Seq[Attribute], + partitionStartIndices: Option[Array[Int]] = None) + extends QueryStageInput { + + override def outputPartitioning: Partitioning = partitionStartIndices.map { + indices => UnknownPartitioning(indices.length) + }.getOrElse(super.outputPartitioning) + + override def doExecute(): RDD[InternalRow] = { + val childRDD = childStage.execute().asInstanceOf[ShuffledRowRDD] + new ShuffledRowRDD(childRDD.dependency, partitionStartIndices) + } +} + +/** A QueryStageInput whose child stage is a BroadcastQueryStage. */ +case class BroadcastQueryStageInput( + childStage: BroadcastQueryStage, + override val output: Seq[Attribute]) + extends QueryStageInput { + + override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + childStage.executeBroadcast() + } + + override def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException( + "BroadcastStageInput does not support the execute() code path.") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index d2d5011bbcb97..8184baf50b042 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -36,107 +36,12 @@ import org.apache.spark.sql.internal.SQLConf * the input partition ordering requirements are met. */ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { - private def defaultNumPreShufflePartitions: Int = conf.numShufflePartitions - - private def targetPostShuffleInputSize: Long = conf.targetPostShuffleInputSize - - private def adaptiveExecutionEnabled: Boolean = conf.adaptiveExecutionEnabled - - private def minNumPostShufflePartitions: Option[Int] = { - val minNumPostShufflePartitions = conf.minNumPostShufflePartitions - if (minNumPostShufflePartitions > 0) Some(minNumPostShufflePartitions) else None - } - - /** - * Adds [[ExchangeCoordinator]] to [[ShuffleExchangeExec]]s if adaptive query execution is enabled - * and partitioning schemes of these [[ShuffleExchangeExec]]s support [[ExchangeCoordinator]]. - */ - private def withExchangeCoordinator( - children: Seq[SparkPlan], - requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = { - val supportsCoordinator = - if (children.exists(_.isInstanceOf[ShuffleExchangeExec])) { - // Right now, ExchangeCoordinator only support HashPartitionings. - children.forall { - case e @ ShuffleExchangeExec(hash: HashPartitioning, _, _) => true - case child => - child.outputPartitioning match { - case hash: HashPartitioning => true - case collection: PartitioningCollection => - collection.partitionings.forall(_.isInstanceOf[HashPartitioning]) - case _ => false - } - } - } else { - // In this case, although we do not have Exchange operators, we may still need to - // shuffle data when we have more than one children because data generated by - // these children may not be partitioned in the same way. - // Please see the comment in withCoordinator for more details. - val supportsDistribution = requiredChildDistributions.forall { dist => - dist.isInstanceOf[ClusteredDistribution] || dist.isInstanceOf[HashClusteredDistribution] - } - children.length > 1 && supportsDistribution - } - - val withCoordinator = - if (adaptiveExecutionEnabled && supportsCoordinator) { - val coordinator = - new ExchangeCoordinator( - targetPostShuffleInputSize, - minNumPostShufflePartitions) - children.zip(requiredChildDistributions).map { - case (e: ShuffleExchangeExec, _) => - // This child is an Exchange, we need to add the coordinator. - e.copy(coordinator = Some(coordinator)) - case (child, distribution) => - // If this child is not an Exchange, we need to add an Exchange for now. - // Ideally, we can try to avoid this Exchange. However, when we reach here, - // there are at least two children operators (because if there is a single child - // and we can avoid Exchange, supportsCoordinator will be false and we - // will not reach here.). Although we can make two children have the same number of - // post-shuffle partitions. Their numbers of pre-shuffle partitions may be different. - // For example, let's say we have the following plan - // Join - // / \ - // Agg Exchange - // / \ - // Exchange t2 - // / - // t1 - // In this case, because a post-shuffle partition can include multiple pre-shuffle - // partitions, a HashPartitioning will not be strictly partitioned by the hashcodes - // after shuffle. So, even we can use the child Exchange operator of the Join to - // have a number of post-shuffle partitions that matches the number of partitions of - // Agg, we cannot say these two children are partitioned in the same way. - // Here is another case - // Join - // / \ - // Agg1 Agg2 - // / \ - // Exchange1 Exchange2 - // / \ - // t1 t2 - // In this case, two Aggs shuffle data with the same column of the join condition. - // After we use ExchangeCoordinator, these two Aggs may not be partitioned in the same - // way. Let's say that Agg1 and Agg2 both have 5 pre-shuffle partitions and 2 - // post-shuffle partitions. It is possible that Agg1 fetches those pre-shuffle - // partitions by using a partitionStartIndices [0, 3]. However, Agg2 may fetch its - // pre-shuffle partitions by using another partitionStartIndices [0, 4]. - // So, Agg1 and Agg2 are actually not co-partitioned. - // - // It will be great to introduce a new Partitioning to represent the post-shuffle - // partitions when one post-shuffle partition includes multiple pre-shuffle partitions. - val targetPartitioning = distribution.createPartitioning(defaultNumPreShufflePartitions) - assert(targetPartitioning.isInstanceOf[HashPartitioning]) - ShuffleExchangeExec(targetPartitioning, child, Some(coordinator)) - } - } else { - // If we do not need ExchangeCoordinator, the original children are returned. - children - } - - withCoordinator - } + private def defaultNumPreShufflePartitions: Int = + if (conf.adaptiveExecutionEnabled) { + conf.maxNumPostShufflePartitions + } else { + conf.numShufflePartitions + } private def ensureDistributionAndOrdering(operator: SparkPlan): SparkPlan = { val requiredChildDistributions: Seq[Distribution] = operator.requiredChildDistribution @@ -189,7 +94,7 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { val defaultPartitioning = distribution.createPartitioning(targetNumPartitions) child match { // If child is an exchange, we replace it with a new one having defaultPartitioning. - case ShuffleExchangeExec(_, c, _) => ShuffleExchangeExec(defaultPartitioning, c) + case ShuffleExchangeExec(_, c) => ShuffleExchangeExec(defaultPartitioning, c) case _ => ShuffleExchangeExec(defaultPartitioning, child) } } @@ -198,15 +103,6 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { } } - // Now, we need to add ExchangeCoordinator if necessary. - // Actually, it is not a good idea to add ExchangeCoordinators while we are adding Exchanges. - // However, with the way that we plan the query, we do not have a place where we have a - // global picture of all shuffle dependencies of a post-shuffle stage. So, we add coordinator - // at here for now. - // Once we finish https://issues.apache.org/jira/browse/SPARK-10665, - // we can first add Exchanges and then add coordinator once we have a DAG of query fragments. - children = withExchangeCoordinator(children, requiredChildDistributions) - // Now that we've performed any necessary shuffles, add sorts to guarantee output orderings: children = children.zip(requiredChildOrderings).map { case (child, requiredOrdering) => // If child.outputOrdering already satisfies the requiredOrdering, we do not need to sort. @@ -295,7 +191,7 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { def apply(plan: SparkPlan): SparkPlan = plan.transformUp { // TODO: remove this after we create a physical operator for `RepartitionByExpression`. - case operator @ ShuffleExchangeExec(upper: HashPartitioning, child, _) => + case operator @ ShuffleExchangeExec(upper: HashPartitioning, child) => child.outputPartitioning match { case lower: HashPartitioning if upper.semanticEquals(lower) => child case _ => operator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala index e4ec76f0b9a1f..99ba2daf36ca1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala @@ -17,60 +17,31 @@ package org.apache.spark.sql.execution.exchange -import java.util.{HashMap => JHashMap, Map => JMap} -import javax.annotation.concurrent.GuardedBy - import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{MapOutputStatistics, ShuffleDependency, SimpleFutureAction} +import org.apache.spark.MapOutputStatistics import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} /** * A coordinator used to determines how we shuffle data between stages generated by Spark SQL. * Right now, the work of this coordinator is to determine the number of post-shuffle partitions * for a stage that needs to fetch shuffle data from one or multiple stages. * - * A coordinator is constructed with three parameters, `numExchanges`, - * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`. - * - `numExchanges` is used to indicated that how many [[ShuffleExchangeExec]]s that will be - * registered to this coordinator. So, when we start to do any actual work, we have a way to - * make sure that we have got expected number of [[ShuffleExchangeExec]]s. + * A coordinator is constructed with two parameters, `targetPostShuffleInputSize`, + * and `minNumPostShufflePartitions`. * - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's * input data size. With this parameter, we can estimate the number of post-shuffle partitions. * This parameter is configured through * `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`. - * - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator - * will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle - * partitions. - * - * The workflow of this coordinator is described as follows: - * - Before the execution of a [[SparkPlan]], for a [[ShuffleExchangeExec]] operator, - * if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator. - * This happens in the `doPrepare` method. - * - Once we start to execute a physical plan, a [[ShuffleExchangeExec]] registered to this - * coordinator will call `postShuffleRDD` to get its corresponding post-shuffle - * [[ShuffledRowRDD]]. - * If this coordinator has made the decision on how to shuffle data, this [[ShuffleExchangeExec]] - * will immediately get its corresponding post-shuffle [[ShuffledRowRDD]]. - * - If this coordinator has not made the decision on how to shuffle data, it will ask those - * registered [[ShuffleExchangeExec]]s to submit their pre-shuffle stages. Then, based on the - * size statistics of pre-shuffle partitions, this coordinator will determine the number of - * post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices - * to a single post-shuffle partition whenever necessary. - * - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered - * [[ShuffleExchangeExec]]s. So, when a [[ShuffleExchangeExec]] calls `postShuffleRDD`, this - * coordinator can lookup the corresponding [[RDD]]. + * - `minNumPostShufflePartitions` is used to make sure that there are at least + * `minNumPostShufflePartitions` post-shuffle partitions. * * The strategy used to determine the number of post-shuffle partitions is described as follows. * To determine the number of post-shuffle partitions, we have a target input size for a - * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages - * corresponding to the registered [[ShuffleExchangeExec]]s, we will do a pass of those statistics - * and pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until - * adding another pre-shuffle partition would cause the size of a post-shuffle partition to be - * greater than the target size. + * post-shuffle partition. Once we have size statistics of all pre-shuffle partitions, we will do + * a pass of those statistics and pack pre-shuffle partitions with continuous indices to a single + * post-shuffle partition until adding another pre-shuffle partition would cause the size of a + * post-shuffle partition to be greater than the target size. * * For example, we have two stages with the following pre-shuffle partition size statistics: * stage 1: [100 MiB, 20 MiB, 100 MiB, 10MiB, 30 MiB] @@ -84,38 +55,9 @@ import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} */ class ExchangeCoordinator( advisoryTargetPostShuffleInputSize: Long, - minNumPostShufflePartitions: Option[Int] = None) + minNumPostShufflePartitions: Int = 1) extends Logging { - // The registered Exchange operators. - private[this] val exchanges = ArrayBuffer[ShuffleExchangeExec]() - - // `lazy val` is used here so that we could notice the wrong use of this class, e.g., all the - // exchanges should be registered before `postShuffleRDD` called first time. If a new exchange is - // registered after the `postShuffleRDD` call, `assert(exchanges.length == numExchanges)` fails - // in `doEstimationIfNecessary`. - private[this] lazy val numExchanges = exchanges.size - - // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator. - private[this] lazy val postShuffleRDDs: JMap[ShuffleExchangeExec, ShuffledRowRDD] = - new JHashMap[ShuffleExchangeExec, ShuffledRowRDD](numExchanges) - - // A boolean that indicates if this coordinator has made decision on how to shuffle data. - // This variable will only be updated by doEstimationIfNecessary, which is protected by - // synchronized. - @volatile private[this] var estimated: Boolean = false - - /** - * Registers a [[ShuffleExchangeExec]] operator to this coordinator. This method is only allowed - * to be called in the `doPrepare` method of a [[ShuffleExchangeExec]] operator. - */ - @GuardedBy("this") - def registerExchange(exchange: ShuffleExchangeExec): Unit = synchronized { - exchanges += exchange - } - - def isEstimated: Boolean = estimated - /** * Estimates partition start indices for post-shuffle partitions based on * mapOutputStatistics provided by all pre-shuffle stages. @@ -125,19 +67,15 @@ class ExchangeCoordinator( // If minNumPostShufflePartitions is defined, it is possible that we need to use a // value less than advisoryTargetPostShuffleInputSize as the target input size of // a post shuffle task. - val targetPostShuffleInputSize = minNumPostShufflePartitions match { - case Some(numPartitions) => - val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum - // The max at here is to make sure that when we have an empty table, we - // only have a single post-shuffle partition. - // There is no particular reason that we pick 16. We just need a number to - // prevent maxPostShuffleInputSize from being set to 0. - val maxPostShuffleInputSize = - math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16) - math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize) - - case None => advisoryTargetPostShuffleInputSize - } + val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum + // The max at here is to make sure that when we have an empty table, we + // only have a single post-shuffle partition. + // There is no particular reason that we pick 16. We just need a number to + // prevent maxPostShuffleInputSize from being set to 0. + val maxPostShuffleInputSize = math.max( + math.ceil(totalPostShuffleInputSize / minNumPostShufflePartitions.toDouble).toLong, 16) + val targetPostShuffleInputSize = + math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize) logInfo( s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " + @@ -189,88 +127,6 @@ class ExchangeCoordinator( partitionStartIndices.toArray } - @GuardedBy("this") - private def doEstimationIfNecessary(): Unit = synchronized { - // It is unlikely that this method will be called from multiple threads - // (when multiple threads trigger the execution of THIS physical) - // because in common use cases, we will create new physical plan after - // users apply operations (e.g. projection) to an existing DataFrame. - // However, if it happens, we have synchronized to make sure only one - // thread will trigger the job submission. - if (!estimated) { - // Make sure we have the expected number of registered Exchange operators. - assert(exchanges.length == numExchanges) - - val newPostShuffleRDDs = new JHashMap[ShuffleExchangeExec, ShuffledRowRDD](numExchanges) - - // Submit all map stages - val shuffleDependencies = ArrayBuffer[ShuffleDependency[Int, InternalRow, InternalRow]]() - val submittedStageFutures = ArrayBuffer[SimpleFutureAction[MapOutputStatistics]]() - var i = 0 - while (i < numExchanges) { - val exchange = exchanges(i) - val shuffleDependency = exchange.prepareShuffleDependency() - shuffleDependencies += shuffleDependency - if (shuffleDependency.rdd.partitions.length != 0) { - // submitMapStage does not accept RDD with 0 partition. - // So, we will not submit this dependency. - submittedStageFutures += - exchange.sqlContext.sparkContext.submitMapStage(shuffleDependency) - } - i += 1 - } - - // Wait for the finishes of those submitted map stages. - val mapOutputStatistics = new Array[MapOutputStatistics](submittedStageFutures.length) - var j = 0 - while (j < submittedStageFutures.length) { - // This call is a blocking call. If the stage has not finished, we will wait at here. - mapOutputStatistics(j) = submittedStageFutures(j).get() - j += 1 - } - - // If we have mapOutputStatistics.length < numExchange, it is because we do not submit - // a stage when the number of partitions of this dependency is 0. - assert(mapOutputStatistics.length <= numExchanges) - - // Now, we estimate partitionStartIndices. partitionStartIndices.length will be the - // number of post-shuffle partitions. - val partitionStartIndices = - if (mapOutputStatistics.length == 0) { - Array.empty[Int] - } else { - estimatePartitionStartIndices(mapOutputStatistics) - } - - var k = 0 - while (k < numExchanges) { - val exchange = exchanges(k) - val rdd = - exchange.preparePostShuffleRDD(shuffleDependencies(k), Some(partitionStartIndices)) - newPostShuffleRDDs.put(exchange, rdd) - - k += 1 - } - - // Finally, we set postShuffleRDDs and estimated. - assert(postShuffleRDDs.isEmpty) - assert(newPostShuffleRDDs.size() == numExchanges) - postShuffleRDDs.putAll(newPostShuffleRDDs) - estimated = true - } - } - - def postShuffleRDD(exchange: ShuffleExchangeExec): ShuffledRowRDD = { - doEstimationIfNecessary() - - if (!postShuffleRDDs.containsKey(exchange)) { - throw new IllegalStateException( - s"The given $exchange is not registered in this coordinator.") - } - - postShuffleRDDs.get(exchange) - } - override def toString: String = { s"coordinator[target post-shuffle partition size: $advisoryTargetPostShuffleInputSize]" } 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 da7b0c6f43fbc..b1adc396e398d 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 @@ -42,8 +42,7 @@ import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordCo */ case class ShuffleExchangeExec( var newPartitioning: Partitioning, - child: SparkPlan, - @transient coordinator: Option[ExchangeCoordinator]) extends Exchange { + child: SparkPlan) extends Exchange { // NOTE: coordinator can be null after serialization/deserialization, // e.g. it can be null on the Executor side @@ -56,14 +55,7 @@ case class ShuffleExchangeExec( ) ++ readMetrics ++ writeMetrics override def nodeName: String = { - val extraInfo = coordinator match { - case Some(exchangeCoordinator) => - s"(coordinator id: ${System.identityHashCode(exchangeCoordinator)})" - case _ => "" - } - - val simpleNodeName = "Exchange" - s"$simpleNodeName$extraInfo" + "Exchange" } override def outputPartitioning: Partitioning = newPartitioning @@ -71,21 +63,6 @@ case class ShuffleExchangeExec( private val serializer: Serializer = new UnsafeRowSerializer(child.output.size, longMetric("dataSize")) - override protected def doPrepare(): Unit = { - // If an ExchangeCoordinator is needed, we register this Exchange operator - // to the coordinator when we do prepare. It is important to make sure - // we register this operator right before the execution instead of register it - // in the constructor because it is possible that we create new instances of - // Exchange operators when we transform the physical plan - // (then the ExchangeCoordinator will hold references of unneeded Exchanges). - // So, we should only call registerExchange just before we start to execute - // the plan. - coordinator match { - case Some(exchangeCoordinator) => exchangeCoordinator.registerExchange(this) - case _ => - } - } - /** * Returns a [[ShuffleDependency]] that will partition rows of its child based on * the partitioning scheme defined in `newPartitioning`. Those partitions of @@ -128,25 +105,32 @@ 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 = coordinator match { - case Some(exchangeCoordinator) => - val shuffleRDD = exchangeCoordinator.postShuffleRDD(this) - assert(shuffleRDD.partitions.length == newPartitioning.numPartitions) - shuffleRDD - case _ => - val shuffleDependency = prepareShuffleDependency() - preparePostShuffleRDD(shuffleDependency) + val shuffleDependency = prepareShuffleDependency() + cachedShuffleRDD = preparePostShuffleRDD(shuffleDependency) + } + cachedShuffleRDD + } + + private var _mapOutputStatistics: MapOutputStatistics = null + + def mapOutputStatistics: MapOutputStatistics = _mapOutputStatistics + + def eagerExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + val shuffleDependency = prepareShuffleDependency() + if (shuffleDependency.rdd.partitions.length != 0) { + // submitMapStage does not accept RDD with 0 partition. + // So, we will not submit this dependency. + val submittedStageFuture = sqlContext.sparkContext.submitMapStage(shuffleDependency) + _mapOutputStatistics = submittedStageFuture.get() } + cachedShuffleRDD = preparePostShuffleRDD(shuffleDependency) } cachedShuffleRDD } } object ShuffleExchangeExec { - def apply(newPartitioning: Partitioning, child: SparkPlan): ShuffleExchangeExec = { - ShuffleExchangeExec(newPartitioning, child, coordinator = Option.empty[ExchangeCoordinator]) - } - /** * Determines whether records must be defensively copied before being sent to the shuffle. * Several of Spark's shuffle components will buffer deserialized Java objects in memory. The diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index 45954f21c5925..a656a2f53e0a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -227,26 +227,26 @@ class SQLAppStatusListener( } } + private def toStoredNodes(nodes: Seq[SparkPlanGraphNode]): Seq[SparkPlanGraphNodeWrapper] = { + nodes.map { + case cluster: SparkPlanGraphCluster => + val storedCluster = new SparkPlanGraphClusterWrapper( + cluster.id, + cluster.name, + cluster.desc, + toStoredNodes(cluster.nodes), + cluster.metrics) + new SparkPlanGraphNodeWrapper(null, storedCluster) + + case node => + new SparkPlanGraphNodeWrapper(node, null) + } + } + private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = { val SparkListenerSQLExecutionStart(executionId, description, details, physicalPlanDescription, sparkPlanInfo, time) = event - def toStoredNodes(nodes: Seq[SparkPlanGraphNode]): Seq[SparkPlanGraphNodeWrapper] = { - nodes.map { - case cluster: SparkPlanGraphCluster => - val storedCluster = new SparkPlanGraphClusterWrapper( - cluster.id, - cluster.name, - cluster.desc, - toStoredNodes(cluster.nodes), - cluster.metrics) - new SparkPlanGraphNodeWrapper(null, storedCluster) - - case node => - new SparkPlanGraphNodeWrapper(node, null) - } - } - val planGraph = SparkPlanGraph(sparkPlanInfo) val sqlPlanMetrics = planGraph.allNodes.flatMap { node => node.metrics.map { metric => (metric.accumulatorId, metric) } @@ -267,6 +267,27 @@ class SQLAppStatusListener( update(exec) } + private def onAdaptiveExecutionUpdate(event: SparkListenerSQLAdaptiveExecutionUpdate): Unit = { + val SparkListenerSQLAdaptiveExecutionUpdate(executionId, + physicalPlanDescription, sparkPlanInfo) = event + + val planGraph = SparkPlanGraph(sparkPlanInfo) + val sqlPlanMetrics = planGraph.allNodes.flatMap { node => + node.metrics.map { metric => (metric.accumulatorId, metric) } + }.toMap.values.toList + + val graphToStore = new SparkPlanGraphWrapper( + executionId, + toStoredNodes(planGraph.nodes), + planGraph.edges) + kvstore.write(graphToStore) + + val exec = getOrCreateExecution(executionId) + exec.physicalPlanDescription = physicalPlanDescription + exec.metrics = sqlPlanMetrics + update(exec) + } + private def onExecutionEnd(event: SparkListenerSQLExecutionEnd): Unit = { val SparkListenerSQLExecutionEnd(executionId, time) = event Option(liveExecutions.get(executionId)).foreach { exec => @@ -295,6 +316,7 @@ class SQLAppStatusListener( override def onOtherEvent(event: SparkListenerEvent): Unit = event match { case e: SparkListenerSQLExecutionStart => onExecutionStart(e) + case e: SparkListenerSQLAdaptiveExecutionUpdate => onAdaptiveExecutionUpdate(e) case e: SparkListenerSQLExecutionEnd => onExecutionEnd(e) case e: SparkListenerDriverAccumUpdates => onDriverAccumUpdates(e) case _ => // Ignore diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 03d75c4c1b82f..eb1e44570ea89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -37,6 +37,13 @@ case class SparkListenerSQLExecutionStart( time: Long) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerSQLAdaptiveExecutionUpdate( + executionId: Long, + physicalPlanDescription: String, + sparkPlanInfo: SparkPlanInfo) + extends SparkListenerEvent + @DeveloperApi case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) extends SparkListenerEvent { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index e57d080dadf78..15b4acfb662b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -96,6 +96,18 @@ object SparkPlanGraph { case "InputAdapter" => buildSparkPlanGraphNode( planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) + case "QueryStage" | "BroadcastQueryStage" | "ResultQueryStage" | "ShuffleQueryStage" => + if (exchanges.contains(planInfo.children.head)) { + // Point to the re-used exchange + val node = exchanges(planInfo.children.head) + edges += SparkPlanGraphEdge(node.id, parent.id) + } else { + buildSparkPlanGraphNode( + planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) + } + case "QueryStageInput" | "ShuffleQueryStageInput" | "BroadcastQueryStageInput" => + buildSparkPlanGraphNode( + planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) case "Subquery" if subgraph != null => // Subquery should not be included in WholeStageCodegen buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, parent, null, exchanges) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index c90b15814a534..46d83061b3b8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1280,7 +1280,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val agg = cp.groupBy('id % 2).agg(count('id)) agg.queryExecution.executedPlan.collectFirst { - case ShuffleExchangeExec(_, _: RDDScanExec, _) => + case ShuffleExchangeExec(_, _: RDDScanExec) => case BroadcastExchangeExec(_, _: RDDScanExec) => }.foreach { _ => fail( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 4a439940beb74..caa6fc67434bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.{MapOutputStatistics, SparkConf, SparkFunSuite} import org.apache.spark.sql._ +import org.apache.spark.sql.execution.adaptive.ShuffleQueryStageInput import org.apache.spark.sql.execution.exchange.{ExchangeCoordinator, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -204,7 +205,11 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } test("test estimatePartitionStartIndices and enforce minimal number of reducers") { +<<<<<<< HEAD val coordinator = new ExchangeCoordinator(100L, Some(2)) +======= + val coordinator = new ExchangeCoordinator(100L, 2) +>>>>>>> d422996... Add QueryStage and the framework for adaptive execution { // The minimal number of post-shuffle partitions is not enforced because @@ -263,7 +268,12 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { .setMaster("local[*]") .setAppName("test") .set("spark.ui.enabled", "false") +<<<<<<< HEAD .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") +======= + .set("spark.driver.allowMultipleContexts", "true") + .set(SQLConf.SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS.key, "5") +>>>>>>> d422996... Add QueryStage and the framework for adaptive execution .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( @@ -273,7 +283,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { case Some(numPartitions) => sparkConf.set(SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS.key, numPartitions.toString) case None => - sparkConf.set(SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS.key, "-1") + sparkConf.set(SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS.key, "1") } val spark = SparkSession.builder() @@ -303,25 +313,21 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val exchanges = agg.queryExecution.executedPlan.collect { - case e: ShuffleExchangeExec => e + val queryStageInputs = agg.queryExecution.executedPlan.collect { + case q: ShuffleQueryStageInput => q } - assert(exchanges.length === 1) + assert(queryStageInputs.length === 1) minNumPostShufflePartitions match { case Some(numPartitions) => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 5) - case o => + queryStageInputs.foreach { q => + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 5) } case None => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 3) - case o => + queryStageInputs.foreach { q => + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 3) } } } @@ -354,25 +360,21 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val exchanges = join.queryExecution.executedPlan.collect { - case e: ShuffleExchangeExec => e + val queryStageInputs = join.queryExecution.executedPlan.collect { + case q: ShuffleQueryStageInput => q } - assert(exchanges.length === 2) + assert(queryStageInputs.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 5) - case o => + queryStageInputs.foreach { q => + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 5) } case None => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 2) - case o => + queryStageInputs.foreach { q => + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 2) } } } @@ -410,26 +412,26 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val exchanges = join.queryExecution.executedPlan.collect { - case e: ShuffleExchangeExec => e + val queryStageInputs = join.queryExecution.executedPlan.collect { + case q: ShuffleQueryStageInput => q } - assert(exchanges.length === 4) + assert(queryStageInputs.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 5) - case o => + queryStageInputs.foreach { q => + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 5) } case None => - assert(exchanges.forall(_.coordinator.isDefined)) - assert(exchanges.map(_.outputPartitioning.numPartitions).toSet === Set(2, 3)) + queryStageInputs.foreach { q => + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 2) + } } } - withSparkSession(test, 6644, minNumPostShufflePartitions) + withSparkSession(test, 16384, minNumPostShufflePartitions) } test(s"determining the number of reducers: complex query 2$testNameNote") { @@ -462,26 +464,26 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val exchanges = join.queryExecution.executedPlan.collect { - case e: ShuffleExchangeExec => e + val queryStageInputs = join.queryExecution.executedPlan.collect { + case q: ShuffleQueryStageInput => q } - assert(exchanges.length === 3) + assert(queryStageInputs.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 5) - case o => + queryStageInputs.foreach { q => + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 5) } case None => - assert(exchanges.forall(_.coordinator.isDefined)) - assert(exchanges.map(_.outputPartitioning.numPartitions).toSet === Set(5, 3)) + queryStageInputs.foreach { q => + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 3) + } } } - withSparkSession(test, 6144, minNumPostShufflePartitions) + withSparkSession(test, 12000, minNumPostShufflePartitions) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 142ab6170a734..c97041a8f341c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -411,8 +411,7 @@ class PlannerSuite extends SharedSQLContext { val inputPlan = ShuffleExchangeExec( partitioning, - DummySparkPlan(outputPartitioning = partitioning), - None) + DummySparkPlan(outputPartitioning = partitioning)) val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) if (outputPlan.collect { case e: ShuffleExchangeExec => true }.size == 2) { @@ -427,8 +426,7 @@ class PlannerSuite extends SharedSQLContext { val inputPlan = ShuffleExchangeExec( partitioning, - DummySparkPlan(outputPartitioning = partitioning), - None) + DummySparkPlan(outputPartitioning = partitioning)) val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) if (outputPlan.collect { case e: ShuffleExchangeExec => true }.size == 1) { @@ -462,8 +460,7 @@ class PlannerSuite extends SharedSQLContext { DummySparkPlan( children = DummySparkPlan(outputPartitioning = childPartitioning) :: Nil, requiredChildDistribution = Seq(distribution), - requiredChildOrdering = Seq(Seq.empty)), - None) + requiredChildOrdering = Seq(Seq.empty))) val inputPlan = SortMergeJoinExec( Literal(1) :: Nil, From 4a9d05402ac1d6a04e88d1171c96facd3b4b5011 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Tue, 10 Apr 2018 14:54:06 +0800 Subject: [PATCH 02/21] update style --- .../execution/ExchangeCoordinatorSuite.scala | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index caa6fc67434bc..7769b03ec5313 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -320,14 +320,14 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { minNumPostShufflePartitions match { case Some(numPartitions) => queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 5) + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 5) } case None => queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 3) + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 3) } } } @@ -367,14 +367,14 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { minNumPostShufflePartitions match { case Some(numPartitions) => queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 5) + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 5) } case None => queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 2) + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 2) } } } @@ -419,14 +419,14 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { minNumPostShufflePartitions match { case Some(numPartitions) => queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 5) + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 5) } case None => queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 2) + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 2) } } } @@ -471,14 +471,14 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { minNumPostShufflePartitions match { case Some(numPartitions) => queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 5) + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 5) } case None => queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 3) + assert(q.partitionStartIndices.isDefined) + assert(q.outputPartitioning.numPartitions === 3) } } } From 3487eb80fdef71007b13678221f4c01b85836135 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 26 Apr 2018 09:22:37 +0800 Subject: [PATCH 03/21] set correct execution Id for broadcast query stage (#50) --- .../apache/spark/sql/execution/adaptive/QueryStage.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala index 83d34681c7c4f..465108b405919 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala @@ -53,16 +53,21 @@ abstract class QueryStage extends UnaryExecNode { * blocking on one child stage. */ def executeChildStages(): Unit = { + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + // Handle broadcast stages val broadcastQueryStages: Seq[BroadcastQueryStage] = child.collect { case bqs: BroadcastQueryStageInput => bqs.childStage } val broadcastFutures = broadcastQueryStages.map { queryStage => - Future { queryStage.prepareBroadcast() }(QueryStage.executionContext) + Future { + SQLExecution.withExecutionId(sqlContext.sparkContext, executionId) { + queryStage.prepareBroadcast() + } + }(QueryStage.executionContext) } // Submit shuffle stages - val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) val shuffleQueryStages: Seq[ShuffleQueryStage] = child.collect { case sqs: ShuffleQueryStageInput => sqs.childStage } From 7df45f8f8f30738a0e93c446534d50ee5a3173be Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Fri, 11 Jan 2019 14:20:17 +0800 Subject: [PATCH 04/21] Add comments to PlanQueryStage and QueryStage --- .../apache/spark/sql/execution/adaptive/PlanQueryStage.scala | 4 ++-- .../org/apache/spark/sql/execution/adaptive/QueryStage.scala | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala index ab2b6e9dfdec1..9901fb03321f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ShuffleExchangeExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -30,7 +29,8 @@ import org.apache.spark.sql.types.StructType /** * Divide the spark plan into multiple QueryStages. For each Exchange in the plan, it adds a * QueryStage and a QueryStageInput. If reusing Exchange is enabled, it finds duplicated exchanges - * and uses the same QueryStage for all the references. + * and uses the same QueryStage for all the references. Note this rule must be run after + * EnsureRequirements rule. */ case class PlanQueryStage(conf: SQLConf) extends Rule[SparkPlan] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala index 465108b405919..92b94da8b8119 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala @@ -99,6 +99,8 @@ abstract class QueryStage extends UnaryExecNode { val queryStageInputs: Seq[ShuffleQueryStageInput] = child.collect { case input: ShuffleQueryStageInput => input } + // mapOutputStatistics can be null if the childStage's RDD has 0 partition. In that case, we + // don't submit that stage and mapOutputStatistics is null. val childMapOutputStatistics = queryStageInputs.map(_.childStage.mapOutputStatistics) .filter(_ != null).toArray if (childMapOutputStatistics.length > 0) { From a83967c46ba6daf9c6dcd73af7faec52e0513d1e Mon Sep 17 00:00:00 2001 From: JkSelf Date: Fri, 23 Nov 2018 13:26:38 +0800 Subject: [PATCH 05/21] Avoid the prepareExecuteStage#QueryStage method is executed multi-times when call executeCollect, executeToIterator and executeTake action multi-times (#70) * Avoid the prepareExecuteStage#QueryStage method is executed multi-times when call executeCollect, executeToIterator and executeTake action multi-times * only add the check in prepareExecuteStage method to avoid duplicate check in other methods * small fix --- .../apache/spark/sql/execution/adaptive/QueryStage.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala index 92b94da8b8119..b21fc21397272 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala @@ -85,12 +85,18 @@ abstract class QueryStage extends UnaryExecNode { Future.sequence(shuffleStageFutures)(implicitly, QueryStage.executionContext), Duration.Inf) } + private var prepared = false + /** * Before executing the plan in this query stage, we execute all child stages, optimize the plan * in this stage and determine the reducer number based on the child stages' statistics. Finally * we do a codegen for this query stage and update the UI with the new plan. */ - def prepareExecuteStage(): Unit = { + def prepareExecuteStage(): Unit = synchronized { + // Ensure the prepareExecuteStage method only be executed once. + if (prepared) { + return + } // 1. Execute childStages executeChildStages() // It is possible to optimize this stage's plan here based on the child stages' statistics. @@ -126,6 +132,7 @@ abstract class QueryStage extends UnaryExecNode { queryExecution.toString, SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan))) } + prepared = true } // Caches the created ShuffleRowRDD so we can reuse that. From 63fece969fd04ed57e0ab4cd25f77225f4fc6174 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Mon, 14 Jan 2019 11:37:18 +0800 Subject: [PATCH 06/21] fix build --- .../apache/spark/sql/execution/QueryExecution.scala | 1 - .../spark/sql/execution/adaptive/QueryStage.scala | 12 +++++++----- .../sql/execution/adaptive/QueryStageInput.scala | 10 ++++++---- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index b5e7c499c6ed1..f1044e1a32f98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -110,7 +110,6 @@ class QueryExecution( ReuseSubquery(sparkSession.sessionState.conf)) protected def adaptivePreparations: Seq[Rule[SparkPlan]] = Seq( - python.ExtractPythonUDFs, PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala index b21fc21397272..a896d83671cbe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala @@ -61,7 +61,7 @@ abstract class QueryStage extends UnaryExecNode { } val broadcastFutures = broadcastQueryStages.map { queryStage => Future { - SQLExecution.withExecutionId(sqlContext.sparkContext, executionId) { + SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) { queryStage.prepareBroadcast() } }(QueryStage.executionContext) @@ -73,7 +73,7 @@ abstract class QueryStage extends UnaryExecNode { } val shuffleStageFutures = shuffleQueryStages.map { queryStage => Future { - SQLExecution.withExecutionId(sqlContext.sparkContext, executionId) { + SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) { queryStage.execute() } }(QueryStage.executionContext) @@ -170,11 +170,13 @@ abstract class QueryStage extends UnaryExecNode { override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - builder: StringBuilder, + append: String => Unit, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): StringBuilder = { - child.generateTreeString(depth, lastChildren, builder, verbose, "*") + addSuffix: Boolean = false, + maxFields: Int): Unit = { + child.generateTreeString( + depth, lastChildren, append, verbose, "", false, maxFields) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala index 887f815b6117a..890add8ea9caf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala @@ -58,11 +58,13 @@ abstract class QueryStageInput extends LeafExecNode { override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - builder: StringBuilder, + append: String => Unit, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): StringBuilder = { - childStage.generateTreeString(depth, lastChildren, builder, verbose, "*") + addSuffix: Boolean = false, + maxFields: Int): Unit = { + childStage.generateTreeString( + depth, lastChildren, append, verbose, "", false, maxFields) } } @@ -84,7 +86,7 @@ case class ShuffleQueryStageInput( override def doExecute(): RDD[InternalRow] = { val childRDD = childStage.execute().asInstanceOf[ShuffledRowRDD] - new ShuffledRowRDD(childRDD.dependency, partitionStartIndices) + new ShuffledRowRDD(childRDD.dependency, childStage.child.metrics, partitionStartIndices) } } From 52c76166cc6120dd03e78006aa2cbda3b6d21713 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Mon, 14 Jan 2019 12:00:06 +0800 Subject: [PATCH 07/21] fix build2 --- .../spark/sql/execution/ExchangeCoordinatorSuite.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 7769b03ec5313..e107db62ccaf7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -205,11 +205,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } test("test estimatePartitionStartIndices and enforce minimal number of reducers") { -<<<<<<< HEAD - val coordinator = new ExchangeCoordinator(100L, Some(2)) -======= val coordinator = new ExchangeCoordinator(100L, 2) ->>>>>>> d422996... Add QueryStage and the framework for adaptive execution { // The minimal number of post-shuffle partitions is not enforced because @@ -268,12 +264,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { .setMaster("local[*]") .setAppName("test") .set("spark.ui.enabled", "false") -<<<<<<< HEAD .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") -======= - .set("spark.driver.allowMultipleContexts", "true") - .set(SQLConf.SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS.key, "5") ->>>>>>> d422996... Add QueryStage and the framework for adaptive execution .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( From 1081a3f7676e13113d4d1f1e22757226a300ed93 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Tue, 15 Jan 2019 14:29:47 +0800 Subject: [PATCH 08/21] Fix test error --- .../spark/sql/execution/ExchangeCoordinatorSuite.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index e107db62ccaf7..a971086744c94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -264,7 +264,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { .setMaster("local[*]") .setAppName("test") .set("spark.ui.enabled", "false") - .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + .set(SQLConf.SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( @@ -484,8 +484,10 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val df = spark.range(1).selectExpr("id AS key", "id AS value") val resultDf = df.join(df, "key").join(df, "key") val sparkPlan = resultDf.queryExecution.executedPlan - assert(sparkPlan.collect { case p: ReusedExchangeExec => p }.length == 1) - assert(sparkPlan.collect { case p @ ShuffleExchangeExec(_, _, Some(c)) => p }.length == 3) + val queryStageInputs = sparkPlan.collect { case p: ShuffleQueryStageInput => p } + assert(queryStageInputs.length === 3) + assert(queryStageInputs(0).childStage === queryStageInputs(1).childStage) + assert(queryStageInputs(1).childStage === queryStageInputs(2).childStage) checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) } withSparkSession(test, 4, None) From 4a2311cd1003548c4f38534c191b3b4517b65d3b Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Tue, 15 Jan 2019 17:47:38 +0800 Subject: [PATCH 09/21] update comments --- .../spark/sql/execution/adaptive/PlanQueryStage.scala | 4 +++- .../spark/sql/execution/adaptive/QueryStageInput.scala | 10 +++++++--- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala index 9901fb03321f0..0f7ab622b75b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala @@ -30,7 +30,9 @@ import org.apache.spark.sql.types.StructType * Divide the spark plan into multiple QueryStages. For each Exchange in the plan, it adds a * QueryStage and a QueryStageInput. If reusing Exchange is enabled, it finds duplicated exchanges * and uses the same QueryStage for all the references. Note this rule must be run after - * EnsureRequirements rule. + * EnsureRequirements rule. The rule divides the plan into multiple sub-trees as QueryStageInput + * is a leaf node. Transforming the plan after applying this rule will only transform node in a + * sub-tree. */ case class PlanQueryStage(conf: SQLConf) extends Rule[SparkPlan] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala index 890add8ea9caf..b8d68d34c5bc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala @@ -25,9 +25,13 @@ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partition import org.apache.spark.sql.execution._ /** - * QueryStageInput is the leaf node of a QueryStage and is used to hide its child stage. It gets - * the result of its child stage and serves it as the input of the QueryStage. A QueryStage knows - * its child stages by collecting all the QueryStageInputs. + * QueryStageInput is the leaf node of a QueryStage and serves as its input. It is responsible for + * changing the output partition based on the need of its QueryStage. It gets the ShuffledRowRDD + * from its child stage and creates a new ShuffledRowRDD with different partitions by specifying + * an optional array of partition start indices. For example, a ShuffledQueryStage can be reused + * by two different QueryStages. One QueryStageInput can let the first task read partition 0 to 3, + * while in another stage, the QueryStageInput can let the first task read partition 0 to 1. + * A QueryStage knows its child stages by collecting all the QueryStageInputs. */ abstract class QueryStageInput extends LeafExecNode { From 2c55985596a38b2177bcfc1d80f86aae65099d5c Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Tue, 15 Jan 2019 22:48:55 +0800 Subject: [PATCH 10/21] update comments --- .../spark/sql/execution/adaptive/QueryStage.scala | 5 ++++- .../sql/execution/adaptive/QueryStageInput.scala | 12 ++++++------ 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala index a896d83671cbe..617f80fde6ff6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala @@ -33,7 +33,10 @@ import org.apache.spark.util.ThreadUtils /** * In adaptive execution mode, an execution plan is divided into multiple QueryStages. Each - * QueryStage is a sub-tree that runs in a single stage. + * QueryStage is a sub-tree that runs in a single stage. Before executing current stage, we will + * first submit all its child stages, wait for their completions and collect their statistics. + * Based on the collected data, we can potentially optimize the execution plan in current stage, + * change the number of reducer and do other optimizations. */ abstract class QueryStage extends UnaryExecNode { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala index b8d68d34c5bc6..8c33e83a91d9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala @@ -25,13 +25,13 @@ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partition import org.apache.spark.sql.execution._ /** - * QueryStageInput is the leaf node of a QueryStage and serves as its input. It is responsible for - * changing the output partition based on the need of its QueryStage. It gets the ShuffledRowRDD + * QueryStageInput is the leaf node of a QueryStage and serves as its input. A QueryStage knows + * its child stages by collecting all the QueryStageInputs. For a ShuffleQueryStageInput, it + * controls how to read the ShuffledRowRDD generated by its child stage. It gets the ShuffledRowRDD * from its child stage and creates a new ShuffledRowRDD with different partitions by specifying - * an optional array of partition start indices. For example, a ShuffledQueryStage can be reused - * by two different QueryStages. One QueryStageInput can let the first task read partition 0 to 3, - * while in another stage, the QueryStageInput can let the first task read partition 0 to 1. - * A QueryStage knows its child stages by collecting all the QueryStageInputs. + * an array of partition start indices. For example, a ShuffledQueryStage can be reused by two + * different QueryStages. One QueryStageInput can let the first task read partition 0 to 3, while + * in another stage, the QueryStageInput can let the first task read partition 0 to 1. */ abstract class QueryStageInput extends LeafExecNode { From ea93dbf591d1387a959183967acbddc8b748d561 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 22 Jan 2019 17:16:30 +0800 Subject: [PATCH 11/21] simplify QueryStage (#5) * do not re-implement exchange reuse * simplify QueryStage * add comments * new idea * polish * address comments * improve QueryStageTrigger --- .../apache/spark/MapOutputStatistics.scala | 1 - .../spark/sql/execution/QueryExecution.scala | 13 +- .../spark/sql/execution/SparkPlanInfo.scala | 5 +- .../adaptive/AdaptiveSparkPlan.scala | 126 +++++++++ .../execution/adaptive/PlanQueryStage.scala | 80 ++---- .../sql/execution/adaptive/QueryStage.scala | 257 +++++++----------- .../execution/adaptive/QueryStageInput.scala | 111 -------- .../adaptive/QueryStagePlanner.scala | 43 +++ .../adaptive/QueryStageTrigger.scala | 160 +++++++++++ .../ReduceNumShufflePartitions.scala} | 94 +++++-- .../exchange/BroadcastExchangeExec.scala | 2 +- .../exchange/EnsureRequirements.scala | 3 +- .../exchange/ShuffleExchangeExec.scala | 63 ++--- .../sql/execution/ui/SparkPlanGraph.scala | 5 +- .../spark/sql/execution/PlannerSuite.scala | 2 +- ... => ReduceNumShufflePartitionsSuite.scala} | 215 ++++++++++----- 16 files changed, 699 insertions(+), 481 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStagePlanner.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageTrigger.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/{exchange/ExchangeCoordinator.scala => adaptive/ReduceNumShufflePartitions.scala} (61%) rename sql/core/src/test/scala/org/apache/spark/sql/execution/{ExchangeCoordinatorSuite.scala => ReduceNumShufflePartitionsSuite.scala} (68%) diff --git a/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala b/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala index c02e48c9815fa..f8a6f1d0d8cbb 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala @@ -25,4 +25,3 @@ package org.apache.spark * (may be inexact due to use of compressed map statuses) */ private[spark] class MapOutputStatistics(val shuffleId: Int, val bytesByPartitionId: Array[Long]) - extends Serializable diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index f1044e1a32f98..f625581d40533 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -94,11 +94,12 @@ class QueryExecution( * row format conversions as needed. */ protected def prepareForExecution(plan: SparkPlan): SparkPlan = { - if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { - adaptivePreparations.foldLeft(plan) { case (sp, rule) => rule.apply(sp)} + val rules = if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { + adaptivePreparations } else { - preparations.foldLeft(plan) { case (sp, rule) => rule.apply(sp)} + preparations } + rules.foldLeft(plan) { case (sp, rule) => rule.apply(sp)} } /** A sequence of rules that will be applied in order to the physical plan before execution. */ @@ -109,14 +110,16 @@ class QueryExecution( ReuseExchange(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf)) + // With adaptive execution, whole stage codegen will be done inside `QueryStageExecutor`. protected def adaptivePreparations: Seq[Rule[SparkPlan]] = Seq( PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), + ReuseExchange(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf), // PlanQueryStage needs to be the last rule because it divides the plan into multiple sub-trees - // by inserting leaf node QueryStageInput. Transforming the plan after applying this rule will + // by inserting leaf node QueryStage. Transforming the plan after applying this rule will // only transform node in a sub-tree. - PlanQueryStage(sparkSession.sessionState.conf)) + PlanQueryStage(sparkSession)) def simpleString: String = withRedaction { val concat = new StringConcat() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index ca46b1e940e46..53b5fc305a330 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.execution.adaptive.QueryStageInput +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlan, QueryStage} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo import org.apache.spark.sql.internal.SQLConf @@ -53,7 +53,8 @@ private[execution] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val children = plan match { case ReusedExchangeExec(_, child) => child :: Nil - case i: QueryStageInput => i.childStage :: Nil + case a: AdaptiveSparkPlan => a.resultStage.plan :: Nil + case stage: QueryStage => stage.plan :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala new file mode 100644 index 0000000000000..32afea26f239d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala @@ -0,0 +1,126 @@ +/* + * 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 java.util.concurrent.CountDownLatch + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, SparkPlanInfo, SQLExecution} +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate + +/** + * A root node to trigger query stages and execute the query plan adaptively. It incrementally + * updates the query plan when a query stage is materialized and provides accurate runtime + * statistics. + */ +case class AdaptiveSparkPlan(initialPlan: ResultQueryStage, session: SparkSession) + extends LeafExecNode{ + + override def output: Seq[Attribute] = initialPlan.output + + @volatile private var currentQueryStage: QueryStage = initialPlan + @volatile private var error: Throwable = null + private val readyLock = new CountDownLatch(1) + + private def replaceStage(oldStage: QueryStage, newStage: QueryStage): QueryStage = { + if (oldStage.id == newStage.id) { + newStage + } else { + val newPlanForOldStage = oldStage.plan.transform { + case q: QueryStage => replaceStage(q, newStage) + } + oldStage.withNewPlan(newPlanForOldStage) + } + } + + private def createCallback(executionId: Option[Long]): QueryStageTriggerCallback = { + new QueryStageTriggerCallback { + override def onStageUpdated(stage: QueryStage): Unit = { + updateCurrentQueryStage(stage, executionId) + if (stage.isInstanceOf[ResultQueryStage]) readyLock.countDown() + } + + override def onStagePlanningFailed(stage: QueryStage, e: Throwable): Unit = { + error = new RuntimeException( + s""" + |Fail to plan stage ${stage.id}: + |${stage.plan.treeString} + """.stripMargin, e) + readyLock.countDown() + } + + override def onStageMaterializingFailed(stage: QueryStage, e: Throwable): Unit = { + error = new RuntimeException( + s""" + |Fail to materialize stage ${stage.id}: + |${stage.plan.treeString} + """.stripMargin, e) + readyLock.countDown() + } + + override def onError(e: Throwable): Unit = { + error = e + readyLock.countDown() + } + } + } + + private def updateCurrentQueryStage(newStage: QueryStage, executionId: Option[Long]): Unit = { + currentQueryStage = replaceStage(currentQueryStage, newStage) + executionId.foreach { id => + session.sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( + id, + SQLExecution.getQueryExecution(id).toString, + SparkPlanInfo.fromSparkPlan(currentQueryStage))) + } + } + + def resultStage: ResultQueryStage = { + if (readyLock.getCount > 0) { + val sc = session.sparkContext + val executionId = Option(sc.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).map(_.toLong) + val trigger = new QueryStageTrigger(session, createCallback(executionId)) + trigger.start() + trigger.trigger(initialPlan) + readyLock.await() + trigger.stop() + } + + if (error != null) throw error + currentQueryStage.asInstanceOf[ResultQueryStage] + } + + override def executeCollect(): Array[InternalRow] = resultStage.executeCollect() + override def executeTake(n: Int): Array[InternalRow] = resultStage.executeTake(n) + override def executeToIterator(): Iterator[InternalRow] = resultStage.executeToIterator() + override def doExecute(): RDD[InternalRow] = resultStage.execute() + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + append: String => Unit, + verbose: Boolean, + prefix: String = "", + addSuffix: Boolean = false, + maxFields: Int): Unit = { + currentQueryStage.generateTreeString( + depth, lastChildren, append, verbose, "", false, maxFields) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala index 0f7ab622b75b9..86ec58c6c77cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala @@ -17,65 +17,41 @@ package org.apache.spark.sql.execution.adaptive -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ShuffleExchangeExec} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec, ShuffleExchangeExec} /** - * Divide the spark plan into multiple QueryStages. For each Exchange in the plan, it adds a - * QueryStage and a QueryStageInput. If reusing Exchange is enabled, it finds duplicated exchanges - * and uses the same QueryStage for all the references. Note this rule must be run after - * EnsureRequirements rule. The rule divides the plan into multiple sub-trees as QueryStageInput - * is a leaf node. Transforming the plan after applying this rule will only transform node in a - * sub-tree. + * Divide the spark plan into multiple QueryStages. For each Exchange in the plan, it wraps it with + * a [[QueryStage]]. At the end it adds an [[AdaptiveSparkPlan]] at the top, which will drive the + * execution of query stages. */ -case class PlanQueryStage(conf: SQLConf) extends Rule[SparkPlan] { +case class PlanQueryStage(session: SparkSession) extends Rule[SparkPlan] { def apply(plan: SparkPlan): SparkPlan = { - - val newPlan = if (!conf.exchangeReuseEnabled) { - plan.transformUp { - case e: ShuffleExchangeExec => - ShuffleQueryStageInput(ShuffleQueryStage(e), e.output) - case e: BroadcastExchangeExec => - BroadcastQueryStageInput(BroadcastQueryStage(e), e.output) - } - } else { - // Build a hash map using schema of exchanges to avoid O(N*N) sameResult calls. - val stages = mutable.HashMap[StructType, ArrayBuffer[QueryStage]]() - - plan.transformUp { - case exchange: Exchange => - val sameSchema = stages.getOrElseUpdate(exchange.schema, ArrayBuffer[QueryStage]()) - val samePlan = sameSchema.find { s => - exchange.sameResult(s.child) - } - if (samePlan.isDefined) { - // Keep the output of this exchange, the following plans require that to resolve - // attributes. - exchange match { - case e: ShuffleExchangeExec => ShuffleQueryStageInput( - samePlan.get.asInstanceOf[ShuffleQueryStage], exchange.output) - case e: BroadcastExchangeExec => BroadcastQueryStageInput( - samePlan.get.asInstanceOf[BroadcastQueryStage], exchange.output) - } - } else { - val queryStageInput = exchange match { - case e: ShuffleExchangeExec => - ShuffleQueryStageInput(ShuffleQueryStage(e), e.output) - case e: BroadcastExchangeExec => - BroadcastQueryStageInput(BroadcastQueryStage(e), e.output) - } - sameSchema += queryStageInput.childStage - queryStageInput - } - } + var id = 0 + val exchangeToQueryStage = new java.util.IdentityHashMap[Exchange, QueryStage] + val planWithStages = plan.transformUp { + case e: ShuffleExchangeExec => + val queryStage = ShuffleQueryStage(id, e) + id += 1 + exchangeToQueryStage.put(e, queryStage) + queryStage + case e: BroadcastExchangeExec => + val queryStage = BroadcastQueryStage(id, e) + id += 1 + exchangeToQueryStage.put(e, queryStage) + queryStage + // The `ReusedExchangeExec` was added in the rule `ReuseExchange`, via transforming up the + // query plan. This rule also transform up the query plan, so when we hit `ReusedExchangeExec` + // here, the exchange being reused must already be hit before and there should be an entry + // for it in `exchangeToQueryStage`. + case e: ReusedExchangeExec => + val existingQueryStage = exchangeToQueryStage.get(e.child) + assert(existingQueryStage != null, "The exchange being reused should be hit before.") + ReusedQueryStage(existingQueryStage, e.output) } - ResultQueryStage(newPlan) + AdaptiveSparkPlan(ResultQueryStage(id, planWithStages), session) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala index 617f80fde6ff6..0994651c20c19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala @@ -17,159 +17,56 @@ package org.apache.spark.sql.execution.adaptive -import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration.Duration +import scala.concurrent.Future import org.apache.spark.MapOutputStatistics -import org.apache.spark.broadcast +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange._ -import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate -import org.apache.spark.util.ThreadUtils /** - * In adaptive execution mode, an execution plan is divided into multiple QueryStages. Each - * QueryStage is a sub-tree that runs in a single stage. Before executing current stage, we will - * first submit all its child stages, wait for their completions and collect their statistics. - * Based on the collected data, we can potentially optimize the execution plan in current stage, - * change the number of reducer and do other optimizations. + * In adaptive execution mode, an execution plan is divided into multiple QueryStages w.r.t. the + * exchange as boundary. Each QueryStage is a sub-tree that runs in a single Spark stage. */ -abstract class QueryStage extends UnaryExecNode { - - var child: SparkPlan - - // Ignore this wrapper for canonicalizing. - override def doCanonicalize(): SparkPlan = child.canonicalized - - override def output: Seq[Attribute] = child.output - - override def outputPartitioning: Partitioning = child.outputPartitioning - - override def outputOrdering: Seq[SortOrder] = child.outputOrdering +abstract class QueryStage extends LeafExecNode { /** - * Execute childStages and wait until all stages are completed. Use a thread pool to avoid - * blocking on one child stage. + * An id of this query stage which is unique in the entire query plan. */ - def executeChildStages(): Unit = { - val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - - // Handle broadcast stages - val broadcastQueryStages: Seq[BroadcastQueryStage] = child.collect { - case bqs: BroadcastQueryStageInput => bqs.childStage - } - val broadcastFutures = broadcastQueryStages.map { queryStage => - Future { - SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) { - queryStage.prepareBroadcast() - } - }(QueryStage.executionContext) - } - - // Submit shuffle stages - val shuffleQueryStages: Seq[ShuffleQueryStage] = child.collect { - case sqs: ShuffleQueryStageInput => sqs.childStage - } - val shuffleStageFutures = shuffleQueryStages.map { queryStage => - Future { - SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) { - queryStage.execute() - } - }(QueryStage.executionContext) - } - - ThreadUtils.awaitResult( - Future.sequence(broadcastFutures)(implicitly, QueryStage.executionContext), Duration.Inf) - ThreadUtils.awaitResult( - Future.sequence(shuffleStageFutures)(implicitly, QueryStage.executionContext), Duration.Inf) - } - - private var prepared = false + def id: Int /** - * Before executing the plan in this query stage, we execute all child stages, optimize the plan - * in this stage and determine the reducer number based on the child stages' statistics. Finally - * we do a codegen for this query stage and update the UI with the new plan. + * The sub-tree of the query plan that belongs to this query stage. */ - def prepareExecuteStage(): Unit = synchronized { - // Ensure the prepareExecuteStage method only be executed once. - if (prepared) { - return - } - // 1. Execute childStages - executeChildStages() - // It is possible to optimize this stage's plan here based on the child stages' statistics. - - // 2. Determine reducer number - val queryStageInputs: Seq[ShuffleQueryStageInput] = child.collect { - case input: ShuffleQueryStageInput => input - } - // mapOutputStatistics can be null if the childStage's RDD has 0 partition. In that case, we - // don't submit that stage and mapOutputStatistics is null. - val childMapOutputStatistics = queryStageInputs.map(_.childStage.mapOutputStatistics) - .filter(_ != null).toArray - if (childMapOutputStatistics.length > 0) { - val exchangeCoordinator = new ExchangeCoordinator( - conf.targetPostShuffleInputSize, - conf.minNumPostShufflePartitions) - - val partitionStartIndices = - exchangeCoordinator.estimatePartitionStartIndices(childMapOutputStatistics) - child = child.transform { - case ShuffleQueryStageInput(childStage, output, _) => - ShuffleQueryStageInput(childStage, output, Some(partitionStartIndices)) - } - } - - // 3. Codegen and update the UI - child = CollapseCodegenStages(sqlContext.conf).apply(child) - val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - if (executionId != null && executionId.nonEmpty) { - val queryExecution = SQLExecution.getQueryExecution(executionId.toLong) - sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( - executionId.toLong, - queryExecution.toString, - SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan))) - } - prepared = true - } - - // Caches the created ShuffleRowRDD so we can reuse that. - private var cachedRDD: RDD[InternalRow] = null - - def executeStage(): RDD[InternalRow] = child.execute() + def plan: SparkPlan /** - * A QueryStage can be reused like Exchange. It is possible that multiple threads try to submit - * the same QueryStage. Use synchronized to make sure it is executed only once. + * Returns a new query stage with a new plan, which is optimized based on accurate runtime + * statistics. */ - override def doExecute(): RDD[InternalRow] = synchronized { - if (cachedRDD == null) { - prepareExecuteStage() - cachedRDD = executeStage() - } - cachedRDD - } - - override def executeCollect(): Array[InternalRow] = { - prepareExecuteStage() - child.executeCollect() - } - - override def executeToIterator(): Iterator[InternalRow] = { - prepareExecuteStage() - child.executeToIterator() - } - - override def executeTake(n: Int): Array[InternalRow] = { - prepareExecuteStage() - child.executeTake(n) - } + def withNewPlan(newPlan: SparkPlan): QueryStage + /** + * Materialize this QueryStage, to prepare for the execution, like submitting map stages, + * broadcasting data, etc. The caller side can use the returned [[Future]] to wait until this + * stage is ready. + */ + def materialize(): Future[Any] + + override def output: Seq[Attribute] = plan.output + override def outputPartitioning: Partitioning = plan.outputPartitioning + override def outputOrdering: Seq[SortOrder] = plan.outputOrdering + override def executeCollect(): Array[InternalRow] = plan.executeCollect() + override def executeTake(n: Int): Array[InternalRow] = plan.executeTake(n) + override def executeToIterator(): Iterator[InternalRow] = plan.executeToIterator() + override def doExecute(): RDD[InternalRow] = plan.execute() + override def doExecuteBroadcast[T](): Broadcast[T] = plan.executeBroadcast() + + // TODO: maybe we should not hide QueryStage entirely from explain result. override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], @@ -178,7 +75,7 @@ abstract class QueryStage extends UnaryExecNode { prefix: String = "", addSuffix: Boolean = false, maxFields: Int): Unit = { - child.generateTreeString( + plan.generateTreeString( depth, lastChildren, append, verbose, "", false, maxFields) } } @@ -186,56 +83,86 @@ abstract class QueryStage extends UnaryExecNode { /** * The last QueryStage of an execution plan. */ -case class ResultQueryStage(var child: SparkPlan) extends QueryStage +case class ResultQueryStage(id: Int, plan: SparkPlan) extends QueryStage { + + override def materialize(): Future[Any] = { + Future.unit + } + + override def withNewPlan(newPlan: SparkPlan): QueryStage = { + copy(plan = newPlan) + } +} /** - * A shuffle QueryStage whose child is a ShuffleExchange. + * A shuffle QueryStage whose child is a ShuffleExchangeExec. */ -case class ShuffleQueryStage(var child: SparkPlan) extends QueryStage { +case class ShuffleQueryStage(id: Int, plan: ShuffleExchangeExec) extends QueryStage { - protected var _mapOutputStatistics: MapOutputStatistics = null - - def mapOutputStatistics: MapOutputStatistics = _mapOutputStatistics + override def withNewPlan(newPlan: SparkPlan): QueryStage = { + copy(plan = newPlan.asInstanceOf[ShuffleExchangeExec]) + } - override def executeStage(): RDD[InternalRow] = { - child match { - case e: ShuffleExchangeExec => - val result = e.eagerExecute() - _mapOutputStatistics = e.mapOutputStatistics - result - case _ => throw new IllegalArgumentException( - "The child of ShuffleQueryStage must be a ShuffleExchange.") + @transient lazy val mapOutputStatisticsFuture: Future[MapOutputStatistics] = { + if (plan.inputRDD.getNumPartitions == 0) { + // `submitMapStage` does not accept RDD with 0 partition. Here we return null and the caller + // side should take care of it. + Future.successful(null) + } else { + sparkContext.submitMapStage(plan.shuffleDependency) } } + + override def materialize(): Future[Any] = { + mapOutputStatisticsFuture + } } /** * A broadcast QueryStage whose child is a BroadcastExchangeExec. */ -case class BroadcastQueryStage(var child: SparkPlan) extends QueryStage { - override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - child.executeBroadcast() +case class BroadcastQueryStage(id: Int, plan: BroadcastExchangeExec) extends QueryStage { + + override def withNewPlan(newPlan: SparkPlan): QueryStage = { + copy(plan = newPlan.asInstanceOf[BroadcastExchangeExec]) } - private var prepared = false + override def materialize(): Future[Any] = { + plan.relationFuture + } +} - def prepareBroadcast() : Unit = synchronized { - if (!prepared) { - executeChildStages() - child = CollapseCodegenStages(sqlContext.conf).apply(child) - // After child stages are completed, prepare() triggers the broadcast. - prepare() - prepared = true +/** + * A wrapper of QueryStage to indicate that it's reused. Note that this is not a query stage. + */ +case class ReusedQueryStage(child: SparkPlan, output: Seq[Attribute]) extends UnaryExecNode { + + // Ignore this wrapper for canonicalizing. + override def doCanonicalize(): SparkPlan = child.canonicalized + + override def doExecute(): RDD[InternalRow] = { + child.execute() + } + + override def doExecuteBroadcast[T](): Broadcast[T] = { + child.executeBroadcast() + } + + // `ReusedQueryStage` can have distinct set of output attribute ids from its child, we need + // to update the attribute ids in `outputPartitioning` and `outputOrdering`. + private lazy val updateAttr: Expression => Expression = { + val originalAttrToNewAttr = AttributeMap(child.output.zip(output)) + e => e.transform { + case attr: Attribute => originalAttrToNewAttr.getOrElse(attr, attr) } } - override def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException( - "BroadcastExchange does not support the execute() code path.") + override def outputPartitioning: Partitioning = child.outputPartitioning match { + case e: Expression => updateAttr(e).asInstanceOf[Partitioning] + case other => other } -} -object QueryStage { - private[execution] val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("adaptive-query-stage")) + override def outputOrdering: Seq[SortOrder] = { + child.outputOrdering.map(updateAttr(_).asInstanceOf[SortOrder]) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala deleted file mode 100644 index 8c33e83a91d9b..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageInput.scala +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.broadcast -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder} -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} -import org.apache.spark.sql.execution._ - -/** - * QueryStageInput is the leaf node of a QueryStage and serves as its input. A QueryStage knows - * its child stages by collecting all the QueryStageInputs. For a ShuffleQueryStageInput, it - * controls how to read the ShuffledRowRDD generated by its child stage. It gets the ShuffledRowRDD - * from its child stage and creates a new ShuffledRowRDD with different partitions by specifying - * an array of partition start indices. For example, a ShuffledQueryStage can be reused by two - * different QueryStages. One QueryStageInput can let the first task read partition 0 to 3, while - * in another stage, the QueryStageInput can let the first task read partition 0 to 1. - */ -abstract class QueryStageInput extends LeafExecNode { - - def childStage: QueryStage - - // Ignore this wrapper for canonicalizing. - override def doCanonicalize(): SparkPlan = childStage.canonicalized - - // Similar to ReusedExchangeExec, two QueryStageInputs can reference to the same childStage. - // QueryStageInput can have distinct set of output attribute ids from its childStage, we need - // to update the attribute ids in outputPartitioning and outputOrdering. - private lazy val updateAttr: Expression => Expression = { - val originalAttrToNewAttr = AttributeMap(childStage.output.zip(output)) - e => e.transform { - case attr: Attribute => originalAttrToNewAttr.getOrElse(attr, attr) - } - } - - override def outputPartitioning: Partitioning = childStage.outputPartitioning match { - case h: HashPartitioning => h.copy(expressions = h.expressions.map(updateAttr)) - case other => other - } - - override def outputOrdering: Seq[SortOrder] = { - childStage.outputOrdering.map(updateAttr(_).asInstanceOf[SortOrder]) - } - - override def generateTreeString( - depth: Int, - lastChildren: Seq[Boolean], - append: String => Unit, - verbose: Boolean, - prefix: String = "", - addSuffix: Boolean = false, - maxFields: Int): Unit = { - childStage.generateTreeString( - depth, lastChildren, append, verbose, "", false, maxFields) - } -} - -/** - * A QueryStageInput whose child stage is a ShuffleQueryStage. It returns a new ShuffledRowRDD - * based on the the child stage's result RDD and the specified partitionStartIndices. If the - * child stage is reused by another ShuffleQueryStageInput, they can return RDDs with different - * partitionStartIndices. - */ -case class ShuffleQueryStageInput( - childStage: ShuffleQueryStage, - override val output: Seq[Attribute], - partitionStartIndices: Option[Array[Int]] = None) - extends QueryStageInput { - - override def outputPartitioning: Partitioning = partitionStartIndices.map { - indices => UnknownPartitioning(indices.length) - }.getOrElse(super.outputPartitioning) - - override def doExecute(): RDD[InternalRow] = { - val childRDD = childStage.execute().asInstanceOf[ShuffledRowRDD] - new ShuffledRowRDD(childRDD.dependency, childStage.child.metrics, partitionStartIndices) - } -} - -/** A QueryStageInput whose child stage is a BroadcastQueryStage. */ -case class BroadcastQueryStageInput( - childStage: BroadcastQueryStage, - override val output: Seq[Attribute]) - extends QueryStageInput { - - override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - childStage.executeBroadcast() - } - - override def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException( - "BroadcastStageInput does not support the execute() code path.") - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStagePlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStagePlanner.scala new file mode 100644 index 0000000000000..0319d146c30dc --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStagePlanner.scala @@ -0,0 +1,43 @@ +/* + * 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.sql.catalyst.rules.{Rule, RuleExecutor} +import org.apache.spark.sql.execution.{CollapseCodegenStages, SparkPlan} +import org.apache.spark.sql.internal.SQLConf + +class QueryStagePlanner(conf: SQLConf) extends RuleExecutor[SparkPlan] { + + override protected def batches: Seq[Batch] = Seq( + Batch("QueryStage Optimization", Once, + AssertChildStagesMaterialized, + ReduceNumShufflePartitions(conf), + CollapseCodegenStages(conf) + ) + ) +} + +// A sanity check rule to make sure we are running `QueryStagePlanner` on a sub-tree of query plan +// with all input stages materialized. +object AssertChildStagesMaterialized extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transform { + case q: QueryStage if !q.materialize().isCompleted => + throw new IllegalArgumentException( + s"The input stages should all be materialize, but ${q.id} is not.") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageTrigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageTrigger.scala new file mode 100644 index 0000000000000..5dd65f42442b2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageTrigger.scala @@ -0,0 +1,160 @@ +/* + * 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.{HashMap, HashSet, ListBuffer} +import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService, Future} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.util.{EventLoop, ThreadUtils} + +/** + * This class triggers [[QueryStage]] bottom-up, apply planner rules for query stages and + * materialize them. It triggers as many query stages as possible at the same time, and triggers + * the parent query stage when all its child stages are materialized. + */ +class QueryStageTrigger(session: SparkSession, callback: QueryStageTriggerCallback) + extends EventLoop[QueryStageTriggerEvent]("QueryStageTrigger") { + + private val stageToParentStages = HashMap.empty[Int, ListBuffer[QueryStage]] + + private val idToUpdatedStage = HashMap.empty[Int, QueryStage] + + private val stageToNumPendingChildStages = HashMap.empty[Int, Int] + + private val submittedStages = HashSet.empty[Int] + + private val readyStages = HashSet.empty[Int] + + private val planner = new QueryStagePlanner(session.sessionState.conf) + + def trigger(stage: QueryStage): Unit = { + post(SubmitStage(stage)) + } + + private implicit def executionContext: ExecutionContextExecutorService = { + QueryStageTrigger.executionContext + } + + override protected def onReceive(event: QueryStageTriggerEvent): Unit = event match { + case SubmitStage(stage) => + // We may submit a query stage multiple times, because of stage reuse. Here we avoid + // re-submitting a query stage. + if (!submittedStages.contains(stage.id)) { + submittedStages += stage.id + val pendingChildStages = stage.plan.collect { + // The stage being submitted may have child stages that are already ready, if the child + // stage is a reused stage. + case stage: QueryStage if !readyStages.contains(stage.id) => stage + } + if (pendingChildStages.isEmpty) { + // This is a leaf stage, or all its child stages are ready, we can plan it now. + post(PlanStage(stage)) + } else { + // This stage has some pending child stages, we store the connection of this stage and + // its child stages, and submit all the child stages, so that we can plan this stage + // later when all its child stages are ready. + stageToNumPendingChildStages(stage.id) = pendingChildStages.length + pendingChildStages.foreach { child => + // a child may have multiple parents, because of query stage reuse. + val parentStages = stageToParentStages.getOrElseUpdate(child.id, new ListBuffer) + parentStages += stage + post(SubmitStage(child)) + } + } + } + + case PlanStage(stage) => + Future { + // planning needs active SparkSession in current thread. + SparkSession.setActiveSession(session) + planner.execute(stage.plan) + }.onComplete { res => + if (res.isSuccess) { + post(StagePlanned(stage, res.get)) + } else { + callback.onStagePlanningFailed(stage, res.failed.get) + stop() + } + } + submittedStages += stage.id + + case StagePlanned(stage, optimizedPlan) => + val newStage = stage.withNewPlan(optimizedPlan) + // We store the new stage with the new query plan after planning, so that later on we can + // update the query plan of its parent stage. + idToUpdatedStage(newStage.id) = newStage + // This stage has optimized its plan, notify the callback about this change. + callback.onStageUpdated(newStage) + + newStage.materialize().onComplete { res => + if (res.isSuccess) { + post(StageReady(stage)) + } else { + callback.onStageMaterializingFailed(newStage, res.failed.get) + stop() + } + } + + case StageReady(stage) => + readyStages += stage.id + stageToParentStages.remove(stage.id).foreach { parentStages => + parentStages.foreach { parent => + val numPendingChildStages = stageToNumPendingChildStages(parent.id) + if (numPendingChildStages == 1) { + stageToNumPendingChildStages.remove(parent.id) + // All its child stages are ready, here we update the query plan via replacing the old + // child stages with new ones that are planned. + val newPlan = parent.plan.transform { + case q: QueryStage => idToUpdatedStage(q.id) + } + // We can plan this stage now. + post(PlanStage(parent.withNewPlan(newPlan))) + } else { + assert(numPendingChildStages > 1) + stageToNumPendingChildStages(parent.id) = numPendingChildStages - 1 + } + } + } + } + + override protected def onError(e: Throwable): Unit = callback.onError(e) +} + +object QueryStageTrigger { + private val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("QueryStageTrigger", 16)) +} + +trait QueryStageTriggerCallback { + def onStageUpdated(stage: QueryStage): Unit + def onStagePlanningFailed(stage: QueryStage, e: Throwable): Unit + def onStageMaterializingFailed(stage: QueryStage, e: Throwable): Unit + def onError(e: Throwable): Unit +} + +sealed trait QueryStageTriggerEvent + +case class SubmitStage(stage: QueryStage) extends QueryStageTriggerEvent + +case class PlanStage(stage: QueryStage) extends QueryStageTriggerEvent + +case class StagePlanned(stage: QueryStage, optimizedPlan: SparkPlan) extends QueryStageTriggerEvent + +case class StageReady(stage: QueryStage) extends QueryStageTriggerEvent diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala similarity index 61% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 99ba2daf36ca1..1f3e85c153dad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -15,26 +15,23 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.exchange +package org.apache.spark.sql.execution.adaptive import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration.Duration import org.apache.spark.MapOutputStatistics -import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.ThreadUtils /** - * A coordinator used to determines how we shuffle data between stages generated by Spark SQL. - * Right now, the work of this coordinator is to determine the number of post-shuffle partitions - * for a stage that needs to fetch shuffle data from one or multiple stages. - * - * A coordinator is constructed with two parameters, `targetPostShuffleInputSize`, - * and `minNumPostShufflePartitions`. - * - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's - * input data size. With this parameter, we can estimate the number of post-shuffle partitions. - * This parameter is configured through - * `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`. - * - `minNumPostShufflePartitions` is used to make sure that there are at least - * `minNumPostShufflePartitions` post-shuffle partitions. + * A rule to adjust the post shuffle partitions based on the map output statistics. * * The strategy used to determine the number of post-shuffle partitions is described as follows. * To determine the number of post-shuffle partitions, we have a target input size for a @@ -53,17 +50,54 @@ import org.apache.spark.internal.Logging * - post-shuffle partition 2: pre-shuffle partition 2 (size 170 MiB) * - post-shuffle partition 3: pre-shuffle partition 3 and 4 (size 50 MiB) */ -class ExchangeCoordinator( - advisoryTargetPostShuffleInputSize: Long, - minNumPostShufflePartitions: Int = 1) - extends Logging { +case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { + + override def apply(plan: SparkPlan): SparkPlan = { + val shuffleMetrics: Seq[MapOutputStatistics] = plan.collect { + case stage: ShuffleQueryStage => + val metricsFuture = stage.mapOutputStatisticsFuture + assert(metricsFuture.isCompleted, "ShuffleQueryStage should already be ready") + ThreadUtils.awaitResult(metricsFuture, Duration.Zero) + } + + val leafNodes = plan.collect { + case s: SparkPlan if s.children.isEmpty => s + } + + if (shuffleMetrics.length == leafNodes.length) { + // ShuffleQueryStage gives null mapOutputStatistics when the input RDD has 0 partitions, + // we should skip it when calculating the `partitionStartIndices`. + val validMetrics = shuffleMetrics.filter(_ != null) + if (validMetrics.nonEmpty) { + val partitionStartIndices = estimatePartitionStartIndices(validMetrics.toArray) + // 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: ShuffleQueryStage => + CoalescedShuffleReaderExec(stage, partitionStartIndices) + } + } else { + plan + } + } else { + // If not all leaf nodes are shuffle 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. + plan + } + } /** * Estimates partition start indices for post-shuffle partitions based on * mapOutputStatistics provided by all pre-shuffle stages. */ - def estimatePartitionStartIndices( + // visible for testing. + private[sql] def estimatePartitionStartIndices( mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = { + val minNumPostShufflePartitions = conf.minNumPostShufflePartitions + 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 // a post shuffle task. @@ -79,7 +113,7 @@ class ExchangeCoordinator( logInfo( s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " + - s"targetPostShuffleInputSize $targetPostShuffleInputSize.") + s"targetPostShuffleInputSize $targetPostShuffleInputSize.") // Make sure we do get the same number of pre-shuffle partitions for those stages. val distinctNumPreShufflePartitions = @@ -126,8 +160,24 @@ class ExchangeCoordinator( partitionStartIndices.toArray } +} + +case class CoalescedShuffleReaderExec( + child: ShuffleQueryStage, + partitionStartIndices: Array[Int]) extends UnaryExecNode { + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = { + UnknownPartitioning(partitionStartIndices.length) + } - override def toString: String = { - s"coordinator[target post-shuffle partition size: $advisoryTargetPostShuffleInputSize]" + private var cachedShuffleRDD: ShuffledRowRDD = null + + override protected def doExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + cachedShuffleRDD = child.plan.createShuffledRDD(Some(partitionStartIndices)) + } + cachedShuffleRDD } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index 703d351bea7c0..18f13cf2eb5ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -66,7 +66,7 @@ case class BroadcastExchangeExec( } @transient - private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { + lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) Future { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 8184baf50b042..126e8e6dd1104 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -24,8 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, - SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf /** 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 b1adc396e398d..987e73e52950f 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 @@ -38,10 +38,10 @@ import org.apache.spark.util.MutablePair import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordComparator} /** - * Performs a shuffle that will result in the desired `newPartitioning`. + * Performs a shuffle that will result in the desired partitioning. */ case class ShuffleExchangeExec( - var newPartitioning: Partitioning, + desiredPartitioning: Partitioning, child: SparkPlan) extends Exchange { // NOTE: coordinator can be null after serialization/deserialization, @@ -58,43 +58,31 @@ case class ShuffleExchangeExec( "Exchange" } - override def outputPartitioning: Partitioning = newPartitioning + override def outputPartitioning: Partitioning = { + desiredPartitioning + } private val serializer: Serializer = new UnsafeRowSerializer(child.output.size, longMetric("dataSize")) + @transient lazy val inputRDD: RDD[InternalRow] = child.execute() + /** - * Returns a [[ShuffleDependency]] that will partition rows of its child based on - * the partitioning scheme defined in `newPartitioning`. Those partitions of - * the returned ShuffleDependency will be the input of shuffle. + * A [[ShuffleDependency]] that will partition rows of its child based on the desired + * partitioning/ Those partitions of the returned ShuffleDependency will be the input of shuffle. */ - private[exchange] def prepareShuffleDependency() - : ShuffleDependency[Int, InternalRow, InternalRow] = { + @transient + lazy val shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow] = { ShuffleExchangeExec.prepareShuffleDependency( - child.execute(), + inputRDD, child.output, - newPartitioning, + outputPartitioning, serializer, writeMetrics) } - /** - * Returns a [[ShuffledRowRDD]] that represents the post-shuffle dataset. - * This [[ShuffledRowRDD]] is created based on a given [[ShuffleDependency]] and an optional - * partition start indices array. If this optional array is defined, the returned - * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array. - */ - private[exchange] def preparePostShuffleRDD( - shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow], - specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = { - // If an array of partition start indices is provided, we need to use this array - // to create the ShuffledRowRDD. Also, we need to update newPartitioning to - // update the number of post-shuffle partitions. - specifiedPartitionStartIndices.foreach { indices => - assert(newPartitioning.isInstanceOf[HashPartitioning]) - newPartitioning = UnknownPartitioning(indices.length) - } - new ShuffledRowRDD(shuffleDependency, readMetrics, specifiedPartitionStartIndices) + def createShuffledRDD(partitionStartIndices: Option[Array[Int]]): ShuffledRowRDD = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices) } /** @@ -105,26 +93,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) { - val shuffleDependency = prepareShuffleDependency() - cachedShuffleRDD = preparePostShuffleRDD(shuffleDependency) - } - cachedShuffleRDD - } - - private var _mapOutputStatistics: MapOutputStatistics = null - - def mapOutputStatistics: MapOutputStatistics = _mapOutputStatistics - - def eagerExecute(): RDD[InternalRow] = { - if (cachedShuffleRDD == null) { - val shuffleDependency = prepareShuffleDependency() - if (shuffleDependency.rdd.partitions.length != 0) { - // submitMapStage does not accept RDD with 0 partition. - // So, we will not submit this dependency. - val submittedStageFuture = sqlContext.sparkContext.submitMapStage(shuffleDependency) - _mapOutputStatistics = submittedStageFuture.get() - } - cachedShuffleRDD = preparePostShuffleRDD(shuffleDependency) + cachedShuffleRDD = createShuffledRDD(None) } cachedShuffleRDD } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index 15b4acfb662b9..113b205367a59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -96,7 +96,7 @@ object SparkPlanGraph { case "InputAdapter" => buildSparkPlanGraphNode( planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) - case "QueryStage" | "BroadcastQueryStage" | "ResultQueryStage" | "ShuffleQueryStage" => + case "BroadcastQueryStage" | "ResultQueryStage" | "ShuffleQueryStage" => if (exchanges.contains(planInfo.children.head)) { // Point to the re-used exchange val node = exchanges(planInfo.children.head) @@ -105,9 +105,6 @@ object SparkPlanGraph { buildSparkPlanGraphNode( planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) } - case "QueryStageInput" | "ShuffleQueryStageInput" | "BroadcastQueryStageInput" => - buildSparkPlanGraphNode( - planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) case "Subquery" if subgraph != null => // Subquery should not be included in WholeStageCodegen buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, parent, null, exchanges) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index c97041a8f341c..e6aa066d8f9db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -448,7 +448,7 @@ class PlannerSuite extends SharedSQLContext { val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) val shuffle = outputPlan.collect { case e: ShuffleExchangeExec => e } assert(shuffle.size === 1) - assert(shuffle.head.newPartitioning === finalPartitioning) + assert(shuffle.head.outputPartitioning === finalPartitioning) } test("Reuse exchanges") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala similarity index 68% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index a88360a90e9eb..70a7ed0e8d1d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -22,12 +22,11 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.{MapOutputStatistics, SparkConf, SparkFunSuite} import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ -import org.apache.spark.sql.execution.adaptive.ShuffleQueryStageInput -import org.apache.spark.sql.execution.exchange.{ExchangeCoordinator, ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.execution.adaptive._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { +class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAll { private var originalActiveSparkSession: Option[SparkSession] = _ private var originalInstantiatedSparkSession: Option[SparkSession] = _ @@ -52,7 +51,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } private def checkEstimation( - coordinator: ExchangeCoordinator, + rule: ReduceNumShufflePartitions, bytesByPartitionIdArray: Array[Array[Long]], expectedPartitionStartIndices: Array[Int]): Unit = { val mapOutputStatistics = bytesByPartitionIdArray.zipWithIndex.map { @@ -60,18 +59,27 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { new MapOutputStatistics(index, bytesByPartitionId) } val estimatedPartitionStartIndices = - coordinator.estimatePartitionStartIndices(mapOutputStatistics) + rule.estimatePartitionStartIndices(mapOutputStatistics) assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) } + private def createReduceNumShufflePartitionsRule( + advisoryTargetPostShuffleInputSize: Long, + minNumPostShufflePartitions: Int = 1): ReduceNumShufflePartitions = { + val conf = new SQLConf().copy( + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE -> advisoryTargetPostShuffleInputSize, + SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS -> minNumPostShufflePartitions) + ReduceNumShufflePartitions(conf) + } + test("test estimatePartitionStartIndices - 1 Exchange") { - val coordinator = new ExchangeCoordinator(100L) + val rule = createReduceNumShufflePartitionsRule(100L) { // All bytes per partition are 0. val bytesByPartitionId = Array[Long](0, 0, 0, 0, 0) val expectedPartitionStartIndices = Array[Int](0) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { @@ -79,40 +87,40 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // 1 post-shuffle partition is needed. val bytesByPartitionId = Array[Long](10, 0, 20, 0, 0) val expectedPartitionStartIndices = Array[Int](0) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { // 2 post-shuffle partitions are needed. val bytesByPartitionId = Array[Long](10, 0, 90, 20, 0) val expectedPartitionStartIndices = Array[Int](0, 3) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { // There are a few large pre-shuffle partitions. val bytesByPartitionId = Array[Long](110, 10, 100, 110, 0) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { // All pre-shuffle partitions are larger than the targeted size. val bytesByPartitionId = Array[Long](100, 110, 100, 110, 110) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { // The last pre-shuffle partition is in a single post-shuffle partition. val bytesByPartitionId = Array[Long](30, 30, 0, 40, 110) val expectedPartitionStartIndices = Array[Int](0, 4) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } } test("test estimatePartitionStartIndices - 2 Exchanges") { - val coordinator = new ExchangeCoordinator(100L) + val rule = createReduceNumShufflePartitionsRule(100L) { // If there are multiple values of the number of pre-shuffle partitions, @@ -123,7 +131,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { Array( new MapOutputStatistics(0, bytesByPartitionId1), new MapOutputStatistics(1, bytesByPartitionId2)) - intercept[AssertionError](coordinator.estimatePartitionStartIndices(mapOutputStatistics)) + intercept[AssertionError](rule.estimatePartitionStartIndices(mapOutputStatistics)) } { @@ -132,7 +140,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) val expectedPartitionStartIndices = Array[Int](0) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -144,7 +152,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 20, 0, 20) val expectedPartitionStartIndices = Array[Int](0) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -155,7 +163,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) val expectedPartitionStartIndices = Array[Int](0, 2, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -166,7 +174,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -177,7 +185,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -188,7 +196,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 60, 0, 110) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -199,14 +207,14 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 60, 70, 110) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } } test("test estimatePartitionStartIndices and enforce minimal number of reducers") { - val coordinator = new ExchangeCoordinator(100L, 2) + val rule = createReduceNumShufflePartitionsRule(100L, 2) { // The minimal number of post-shuffle partitions is not enforced because @@ -215,7 +223,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) val expectedPartitionStartIndices = Array[Int](0) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -226,7 +234,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](5, 10, 0, 10, 5) val expectedPartitionStartIndices = Array[Int](0, 3) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -237,7 +245,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](40, 10, 0, 10, 30) val expectedPartitionStartIndices = Array[Int](0, 1, 3, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -258,7 +266,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { def withSparkSession( f: SparkSession => Unit, - targetNumPostShufflePartitions: Int, + targetPostShuffleInputSize: Int, minNumPostShufflePartitions: Option[Int]): Unit = { val sparkConf = new SparkConf(false) @@ -270,7 +278,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, - targetNumPostShufflePartitions.toString) + targetPostShuffleInputSize.toString) minNumPostShufflePartitions match { case Some(numPartitions) => sparkConf.set(SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS.key, numPartitions.toString) @@ -305,21 +313,21 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val queryStageInputs = agg.queryExecution.executedPlan.collect { - case q: ShuffleQueryStageInput => q + val finalPlan = agg.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader } - assert(queryStageInputs.length === 1) + assert(shuffleReaders.length === 1) minNumPostShufflePartitions match { case Some(numPartitions) => - queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 5) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === numPartitions) } case None => - queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 3) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === 3) } } } @@ -352,21 +360,21 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val queryStageInputs = join.queryExecution.executedPlan.collect { - case q: ShuffleQueryStageInput => q + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader } - assert(queryStageInputs.length === 2) + assert(shuffleReaders.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => - queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 5) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === numPartitions) } case None => - queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 2) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === 2) } } } @@ -404,21 +412,21 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val queryStageInputs = join.queryExecution.executedPlan.collect { - case q: ShuffleQueryStageInput => q + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader } - assert(queryStageInputs.length === 2) + assert(shuffleReaders.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => - queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 5) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === numPartitions) } case None => - queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 2) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === 2) } } } @@ -456,40 +464,111 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val queryStageInputs = join.queryExecution.executedPlan.collect { - case q: ShuffleQueryStageInput => q + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader } - assert(queryStageInputs.length === 2) + assert(shuffleReaders.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => - queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 5) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === numPartitions) } case None => - queryStageInputs.foreach { q => - assert(q.partitionStartIndices.isDefined) - assert(q.outputPartitioning.numPartitions === 3) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === 3) } } } withSparkSession(test, 12000, minNumPostShufflePartitions) } + + test(s"determining the number of reducers: plan already partitioned$testNameNote") { + val test: SparkSession => Unit = { spark: SparkSession => + try { + spark.range(1000).write.bucketBy(30, "id").saveAsTable("t") + // `df1` is hash partitioned by `id`. + val df1 = spark.read.table("t") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = df1.join(df2, col("id") === col("key2")).select(col("id"), col("value2")) + + // Check the answer first. + val expectedAnswer = spark.range(0, 500).selectExpr("id % 500", "id as value") + .union(spark.range(500, 1000).selectExpr("id % 500", "id as value")) + checkAnswer( + join, + expectedAnswer.collect()) + + // Then, let's make sure we do not reduce number of ppst shuffle partitions. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader + } + assert(shuffleReaders.length === 0) + } finally { + spark.sql("drop table t") + } + } + withSparkSession(test, 12000, minNumPostShufflePartitions) + } } test("SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") { val test = { spark: SparkSession => spark.sql("SET spark.sql.exchange.reuse=true") val df = spark.range(1).selectExpr("id AS key", "id AS value") + + // test case 1: a stage has 3 child stages but they are the same stage. + // ResultQueryStage 1 + // ShuffleQueryStage 0 + // ReusedQueryStage 0 + // ReusedQueryStage 0 val resultDf = df.join(df, "key").join(df, "key") - val sparkPlan = resultDf.queryExecution.executedPlan - val queryStageInputs = sparkPlan.collect { case p: ShuffleQueryStageInput => p } - assert(queryStageInputs.length === 3) - assert(queryStageInputs(0).childStage === queryStageInputs(1).childStage) - assert(queryStageInputs(1).childStage === queryStageInputs(2).childStage) + val finalPlan = resultDf.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + assert(finalPlan.collect { case p: ReusedQueryStage => p }.length == 2) + assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 3) checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) + + // test case 2: a stage has 2 parent stages. + // ResultQueryStage 3 + // ShuffleQueryStage 1 + // ShuffleQueryStage 0 + // ShuffleQueryStage 2 + // ReusedQueryStage 0 + val grouped = df.groupBy("key").agg(max("value").as("value")) + val resultDf2 = grouped.groupBy(col("key") + 1).max("value") + .union(grouped.groupBy(col("key") + 2).max("value")) + + val resultStage = resultDf2.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlan].resultStage + + // The result stage has 2 children + val level1Stages = resultStage.plan.collect { case q: QueryStage => q } + assert(level1Stages.length == 2) + + val leafStages = level1Stages.flatMap { stage => + // All of the child stages of result stage have only one child stage. + val children = stage.plan.collect { case q: QueryStage => q } + assert(children.length == 1) + children + } + assert(leafStages.length == 2) + + val reusedStages = level1Stages.flatMap { stage => + stage.plan.collect { case r: ReusedQueryStage => r } + } + assert(reusedStages.length == 1) + + checkAnswer(resultDf2, Row(1, 0) :: Row(2, 0) :: Nil) } withSparkSession(test, 4, None) } From 068ef94e434a06e54d32a5f632747d826e02282e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 31 Jan 2019 13:26:54 +0800 Subject: [PATCH 12/21] insert query stages dynamically (#6) * insert query stages dynamically * add comment * address comments --- .../spark/sql/execution/QueryExecution.scala | 27 +- .../spark/sql/execution/SparkPlanInfo.scala | 2 +- .../adaptive/AdaptiveSparkPlan.scala | 69 ++--- .../adaptive/InsertAdaptiveSparkPlan.scala | 38 +++ .../execution/adaptive/PlanQueryStage.scala | 57 ---- .../sql/execution/adaptive/QueryStage.scala | 12 +- .../adaptive/QueryStageCreator.scala | 255 ++++++++++++++++++ .../adaptive/QueryStageTrigger.scala | 160 ----------- .../AssertChildStagesMaterialized.scala} | 23 +- .../ReduceNumShufflePartitions.scala | 3 +- .../ReduceNumShufflePartitionsSuite.scala | 15 +- 11 files changed, 351 insertions(+), 310 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageTrigger.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/{QueryStagePlanner.scala => rule/AssertChildStagesMaterialized.scala} (62%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/{ => rule}/ReduceNumShufflePartitions.scala (98%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index f625581d40533..37e1e54d8766b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.adaptive.PlanQueryStage +import org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils @@ -94,32 +94,19 @@ class QueryExecution( * row format conversions as needed. */ protected def prepareForExecution(plan: SparkPlan): SparkPlan = { - val rules = if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { - adaptivePreparations - } else { - preparations - } - rules.foldLeft(plan) { case (sp, rule) => rule.apply(sp)} + preparations.foldLeft(plan) { case (sp, rule) => rule.apply(sp) } } /** A sequence of rules that will be applied in order to the physical plan before execution. */ protected def preparations: Seq[Rule[SparkPlan]] = Seq( PlanSubqueries(sparkSession), + ReuseSubquery(sparkSession.sessionState.conf), EnsureRequirements(sparkSession.sessionState.conf), + // `AdaptiveSparkPlan` is a leaf node. If inserted, all the following rules will be no-op as + // the original plan is hidden behind `AdaptiveSparkPlan`. + InsertAdaptiveSparkPlan(sparkSession), CollapseCodegenStages(sparkSession.sessionState.conf), - ReuseExchange(sparkSession.sessionState.conf), - ReuseSubquery(sparkSession.sessionState.conf)) - - // With adaptive execution, whole stage codegen will be done inside `QueryStageExecutor`. - protected def adaptivePreparations: Seq[Rule[SparkPlan]] = Seq( - PlanSubqueries(sparkSession), - EnsureRequirements(sparkSession.sessionState.conf), - ReuseExchange(sparkSession.sessionState.conf), - ReuseSubquery(sparkSession.sessionState.conf), - // PlanQueryStage needs to be the last rule because it divides the plan into multiple sub-trees - // by inserting leaf node QueryStage. Transforming the plan after applying this rule will - // only transform node in a sub-tree. - PlanQueryStage(sparkSession)) + ReuseExchange(sparkSession.sessionState.conf)) def simpleString: String = withRedaction { val concat = new StringConcat() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 53b5fc305a330..9c6d5928259d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -53,7 +53,7 @@ private[execution] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val children = plan match { case ReusedExchangeExec(_, child) => child :: Nil - case a: AdaptiveSparkPlan => a.resultStage.plan :: Nil + case a: AdaptiveSparkPlan => a.finalPlan.plan :: Nil case stage: QueryStage => stage.plan :: Nil case _ => plan.children ++ plan.subqueries } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala index 32afea26f239d..44f6c6f497918 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.adaptive import java.util.concurrent.CountDownLatch +import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow @@ -27,48 +28,33 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, SparkPlanInfo, S import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate /** - * A root node to trigger query stages and execute the query plan adaptively. It incrementally + * A root node to execute the query plan adaptively. It creates query stages, and incrementally * updates the query plan when a query stage is materialized and provides accurate runtime - * statistics. + * data statistics. */ -case class AdaptiveSparkPlan(initialPlan: ResultQueryStage, session: SparkSession) +case class AdaptiveSparkPlan(initialPlan: SparkPlan, session: SparkSession) extends LeafExecNode{ override def output: Seq[Attribute] = initialPlan.output - @volatile private var currentQueryStage: QueryStage = initialPlan + @volatile private var currentPlan: SparkPlan = initialPlan @volatile private var error: Throwable = null - private val readyLock = new CountDownLatch(1) - private def replaceStage(oldStage: QueryStage, newStage: QueryStage): QueryStage = { - if (oldStage.id == newStage.id) { - newStage - } else { - val newPlanForOldStage = oldStage.plan.transform { - case q: QueryStage => replaceStage(q, newStage) - } - oldStage.withNewPlan(newPlanForOldStage) - } - } + // We will release the lock when we finish planning query stages, or we fail to do the planning. + // Getting `resultStage` will be blocked until the lock is release. + // This is better than wait()/notify(), as we can easily check if the computation has completed, + // by calling `readyLock.getCount()`. + private val readyLock = new CountDownLatch(1) private def createCallback(executionId: Option[Long]): QueryStageTriggerCallback = { new QueryStageTriggerCallback { - override def onStageUpdated(stage: QueryStage): Unit = { - updateCurrentQueryStage(stage, executionId) - if (stage.isInstanceOf[ResultQueryStage]) readyLock.countDown() - } - - override def onStagePlanningFailed(stage: QueryStage, e: Throwable): Unit = { - error = new RuntimeException( - s""" - |Fail to plan stage ${stage.id}: - |${stage.plan.treeString} - """.stripMargin, e) - readyLock.countDown() + override def onPlanUpdate(updatedPlan: SparkPlan): Unit = { + updateCurrentPlan(updatedPlan, executionId) + if (updatedPlan.isInstanceOf[ResultQueryStage]) readyLock.countDown() } override def onStageMaterializingFailed(stage: QueryStage, e: Throwable): Unit = { - error = new RuntimeException( + error = new SparkException( s""" |Fail to materialize stage ${stage.id}: |${stage.plan.treeString} @@ -83,35 +69,34 @@ case class AdaptiveSparkPlan(initialPlan: ResultQueryStage, session: SparkSessio } } - private def updateCurrentQueryStage(newStage: QueryStage, executionId: Option[Long]): Unit = { - currentQueryStage = replaceStage(currentQueryStage, newStage) + private def updateCurrentPlan(newPlan: SparkPlan, executionId: Option[Long]): Unit = { + currentPlan = newPlan executionId.foreach { id => session.sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( id, SQLExecution.getQueryExecution(id).toString, - SparkPlanInfo.fromSparkPlan(currentQueryStage))) + SparkPlanInfo.fromSparkPlan(currentPlan))) } } - def resultStage: ResultQueryStage = { + def finalPlan: ResultQueryStage = { if (readyLock.getCount > 0) { val sc = session.sparkContext val executionId = Option(sc.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).map(_.toLong) - val trigger = new QueryStageTrigger(session, createCallback(executionId)) - trigger.start() - trigger.trigger(initialPlan) + val creator = new QueryStageCreator(initialPlan, session, createCallback(executionId)) + creator.start() readyLock.await() - trigger.stop() + creator.stop() } if (error != null) throw error - currentQueryStage.asInstanceOf[ResultQueryStage] + currentPlan.asInstanceOf[ResultQueryStage] } - override def executeCollect(): Array[InternalRow] = resultStage.executeCollect() - override def executeTake(n: Int): Array[InternalRow] = resultStage.executeTake(n) - override def executeToIterator(): Iterator[InternalRow] = resultStage.executeToIterator() - override def doExecute(): RDD[InternalRow] = resultStage.execute() + override def executeCollect(): Array[InternalRow] = finalPlan.executeCollect() + override def executeTake(n: Int): Array[InternalRow] = finalPlan.executeTake(n) + override def executeToIterator(): Iterator[InternalRow] = finalPlan.executeToIterator() + override def doExecute(): RDD[InternalRow] = finalPlan.execute() override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], @@ -120,7 +105,7 @@ case class AdaptiveSparkPlan(initialPlan: ResultQueryStage, session: SparkSessio prefix: String = "", addSuffix: Boolean = false, maxFields: Int): Unit = { - currentQueryStage.generateTreeString( + currentPlan.generateTreeString( depth, lastChildren, append, verbose, "", false, maxFields) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala new file mode 100644 index 0000000000000..4a1297b71feb8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -0,0 +1,38 @@ +/* + * 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.sql.SparkSession +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.command.ExecutedCommandExec + +/** + * This rule wraps the query plan with an [[AdaptiveSparkPlan]], which executes the query plan + * adaptively with runtime data statistics. Note that this rule must be run after + * [[org.apache.spark.sql.execution.exchange.EnsureRequirements]], so that the exchange nodes are + * already inserted. + */ +case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan] { + + override def apply(plan: SparkPlan): SparkPlan = plan match { + case _: ExecutedCommandExec => plan + case _ if session.sessionState.conf.adaptiveExecutionEnabled => + AdaptiveSparkPlan(plan, session.cloneSession()) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala deleted file mode 100644 index 86ec58c6c77cd..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanQueryStage.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.sql.SparkSession -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec, ShuffleExchangeExec} - -/** - * Divide the spark plan into multiple QueryStages. For each Exchange in the plan, it wraps it with - * a [[QueryStage]]. At the end it adds an [[AdaptiveSparkPlan]] at the top, which will drive the - * execution of query stages. - */ -case class PlanQueryStage(session: SparkSession) extends Rule[SparkPlan] { - - def apply(plan: SparkPlan): SparkPlan = { - var id = 0 - val exchangeToQueryStage = new java.util.IdentityHashMap[Exchange, QueryStage] - val planWithStages = plan.transformUp { - case e: ShuffleExchangeExec => - val queryStage = ShuffleQueryStage(id, e) - id += 1 - exchangeToQueryStage.put(e, queryStage) - queryStage - case e: BroadcastExchangeExec => - val queryStage = BroadcastQueryStage(id, e) - id += 1 - exchangeToQueryStage.put(e, queryStage) - queryStage - // The `ReusedExchangeExec` was added in the rule `ReuseExchange`, via transforming up the - // query plan. This rule also transform up the query plan, so when we hit `ReusedExchangeExec` - // here, the exchange being reused must already be hit before and there should be an entry - // for it in `exchangeToQueryStage`. - case e: ReusedExchangeExec => - val existingQueryStage = exchangeToQueryStage.get(e.child) - assert(existingQueryStage != null, "The exchange being reused should be hit before.") - ReusedQueryStage(existingQueryStage, e.output) - } - AdaptiveSparkPlan(ResultQueryStage(id, planWithStages), session) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala index 0994651c20c19..6edd1e4eafb6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala @@ -29,8 +29,9 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange._ /** - * In adaptive execution mode, an execution plan is divided into multiple QueryStages w.r.t. the - * exchange as boundary. Each QueryStage is a sub-tree that runs in a single Spark stage. + * A query stage is an individual sub-tree of a query plan, which can be executed ahead and provide + * accurate data statistics. For example, a sub-tree under shuffle/broadcast node is a query stage. + * Each query stage runs in a single Spark job/stage. */ abstract class QueryStage extends LeafExecNode { @@ -65,6 +66,7 @@ abstract class QueryStage extends LeafExecNode { override def executeToIterator(): Iterator[InternalRow] = plan.executeToIterator() override def doExecute(): RDD[InternalRow] = plan.execute() override def doExecuteBroadcast[T](): Broadcast[T] = plan.executeBroadcast() + override def doCanonicalize(): SparkPlan = plan.canonicalized // TODO: maybe we should not hide QueryStage entirely from explain result. override def generateTreeString( @@ -86,7 +88,7 @@ abstract class QueryStage extends LeafExecNode { case class ResultQueryStage(id: Int, plan: SparkPlan) extends QueryStage { override def materialize(): Future[Any] = { - Future.unit + throw new IllegalStateException("Cannot materialize ResultQueryStage.") } override def withNewPlan(newPlan: SparkPlan): QueryStage = { @@ -95,7 +97,7 @@ case class ResultQueryStage(id: Int, plan: SparkPlan) extends QueryStage { } /** - * A shuffle QueryStage whose child is a ShuffleExchangeExec. + * A shuffle QueryStage whose child is a [[ShuffleExchangeExec]]. */ case class ShuffleQueryStage(id: Int, plan: ShuffleExchangeExec) extends QueryStage { @@ -119,7 +121,7 @@ case class ShuffleQueryStage(id: Int, plan: ShuffleExchangeExec) extends QuerySt } /** - * A broadcast QueryStage whose child is a BroadcastExchangeExec. + * A broadcast QueryStage whose child is a [[BroadcastExchangeExec]]. */ case class BroadcastQueryStage(id: Int, plan: BroadcastExchangeExec) extends QueryStage { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala new file mode 100644 index 0000000000000..56f883f5123f8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala @@ -0,0 +1,255 @@ +/* + * 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.{ExecutionContext, ExecutionContextExecutorService} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{CollapseCodegenStages, SparkPlan} +import org.apache.spark.sql.execution.adaptive.rule.{AssertChildStagesMaterialized, ReduceNumShufflePartitions} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ShuffleExchangeExec} +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.{EventLoop, ThreadUtils} + +/** + * This class dynamically creates [[QueryStage]] bottom-up, optimize the query plan of query stages + * and materialize them. It creates as many query stages as possible at the same time, and + * materialize a query stage when all its child stages are materialized. + * + * To create query stages, we traverse the query tree bottom up. When we hit an exchange node, and + * all the child query stages of this exchange node are materialized, we try to create a new query + * stage for this exchange node. + * + * To create a new query stage, we first optimize the sub-tree of the exchange. After optimization, + * we check the output partitioning of the optimized sub-tree, and see if the exchange node is still + * necessary. + * + * If the exchange node becomes unnecessary, remove it and give up this query stage creation, and + * continue to traverse the query plan tree until we hit the next exchange node. + * + * If the exchange node is still needed, create the query stage and optimize its sub-tree again. + * It's necessary to have both the pre-creation optimization and post-creation optimization, because + * these 2 optimization have different assumptions. For pre-creation optimization, the shuffle node + * may be removed later on and the current sub-tree may be only a part of a query stage, so we don't + * have the big picture of the query stage yet. For post-creation optimization, the query stage is + * created and we have the big picture of the query stage. + * + * After the query stage is optimized, we materialize it asynchronously, and continue to traverse + * the query plan tree to create more query stages. + * + * When a query stage completes materialization, we trigger the process of query stages creation and + * traverse the query plan tree again. + */ +class QueryStageCreator( + initialPlan: SparkPlan, + session: SparkSession, + callback: QueryStageTriggerCallback) + extends EventLoop[QueryStageCreatorEvent]("QueryStageCreator") { + + private def conf = session.sessionState.conf + + private val readyStages = mutable.HashSet.empty[Int] + + private var currentStageId = 0 + + private val stageCache = mutable.HashMap.empty[StructType, mutable.Buffer[(Exchange, QueryStage)]] + + // The optimizer rules that will be applied to a sub-tree of the query plan before the stage is + // created. Note that we may end up not creating the query stage, so the rules here should not + // assume the given sub-plan-tree is the entire query plan of the query stage. For example, if a + // rule want to collect all the child query stages, it should not be put here. + private val preStageCreationOptimizerRules: Seq[Rule[SparkPlan]] = Seq( + AssertChildStagesMaterialized + ) + + // The optimizer rules that will be applied to a sub-tree of the query plan after the stage is + // created. Note that once the stage is created, we will not remove it anymore. If a rule changes + // the output partitioning of the sub-plan-tree, which may help to remove the exchange node, it's + // better to put it in `preStageCreationOptimizerRules`, so that we may create less query stages. + private val postStageCreationOptimizerRules: Seq[Rule[SparkPlan]] = Seq( + ReduceNumShufflePartitions(conf), + CollapseCodegenStages(conf)) + + private var currentPlan = initialPlan + + private implicit def executionContext: ExecutionContextExecutorService = { + QueryStageCreator.executionContext + } + + override protected def onReceive(event: QueryStageCreatorEvent): Unit = event match { + case StartCreation => + // set active session for the event loop thread. + SparkSession.setActiveSession(session) + currentPlan = createQueryStages(initialPlan) + + case MaterializeStage(stage) => + stage.materialize().onComplete { res => + if (res.isSuccess) { + post(StageReady(stage)) + } else { + callback.onStageMaterializingFailed(stage, res.failed.get) + stop() + } + } + + case StageReady(stage) => + if (stage.isInstanceOf[ResultQueryStage]) { + callback.onPlanUpdate(stage) + stop() + } else { + readyStages += stage.id + currentPlan = createQueryStages(currentPlan) + } + } + + override protected def onStart(): Unit = { + post(StartCreation) + } + + private def preStageCreationOptimize(plan: SparkPlan): SparkPlan = { + preStageCreationOptimizerRules.foldLeft(plan) { + case (current, rule) => rule(current) + } + } + + private def postStageCreationOptimize(plan: SparkPlan): SparkPlan = { + postStageCreationOptimizerRules.foldLeft(plan) { + case (current, rule) => rule(current) + } + } + + /** + * Traverse the query plan bottom-up, and creates query stages as many as possible. + */ + private def createQueryStages(plan: SparkPlan): SparkPlan = { + val result = createQueryStages0(plan) + if (result.allChildStagesReady) { + val finalPlan = postStageCreationOptimize(preStageCreationOptimize(result.newPlan)) + post(StageReady(ResultQueryStage(currentStageId, finalPlan))) + finalPlan + } else { + callback.onPlanUpdate(result.newPlan) + result.newPlan + } + } + + /** + * This method is called recursively to traverse the plan tree bottom-up. This method returns two + * information: 1) the new plan after we insert query stages. 2) whether or not the child query + * stages of the new plan are all ready. + * + * if the current plan is an exchange node, and all its child query stages are ready, we try to + * create a new query stage. + */ + private def createQueryStages0(plan: SparkPlan): CreateStageResult = plan match { + case e: Exchange => + val similarStages = stageCache.getOrElseUpdate(e.schema, mutable.Buffer.empty) + similarStages.find(_._1.sameResult(e)) match { + case Some((_, existingStage)) if conf.exchangeReuseEnabled => + CreateStageResult( + newPlan = ReusedQueryStage(existingStage, e.output), + allChildStagesReady = readyStages.contains(existingStage.id)) + + case _ => + val result = createQueryStages0(e.child) + // Try to create a query stage only when all the child query stages are ready. + if (result.allChildStagesReady) { + val optimizedPlan = preStageCreationOptimize(result.newPlan) + e match { + case s: ShuffleExchangeExec => + (s.desiredPartitioning, optimizedPlan.outputPartitioning) match { + case (desired: HashPartitioning, actual: HashPartitioning) + if desired.semanticEquals(actual) => + // This shuffle exchange is unnecessary now, remove it. The reason maybe: + // 1. the child plan has changed its output partitioning after optimization, + // and makes this exchange node unnecessary. + // 2. this exchange node is user specified, which turns out to be unnecessary. + CreateStageResult(newPlan = optimizedPlan, allChildStagesReady = true) + case _ => + val queryStage = createQueryStage(s.copy(child = optimizedPlan)) + similarStages.append(e -> queryStage) + // We've created a new stage, which is obviously not ready yet. + CreateStageResult(newPlan = queryStage, allChildStagesReady = false) + } + + case b: BroadcastExchangeExec => + val queryStage = createQueryStage(b.copy(child = optimizedPlan)) + similarStages.append(e -> queryStage) + // We've created a new stage, which is obviously not ready yet. + CreateStageResult(newPlan = queryStage, allChildStagesReady = false) + } + } else { + CreateStageResult( + newPlan = e.withNewChildren(Seq(result.newPlan)), + allChildStagesReady = false) + } + } + + case q: QueryStage => + CreateStageResult(newPlan = q, allChildStagesReady = readyStages.contains(q.id)) + + case _ => + if (plan.children.isEmpty) { + CreateStageResult(newPlan = plan, allChildStagesReady = true) + } else { + val results = plan.children.map(createQueryStages0) + CreateStageResult( + newPlan = plan.withNewChildren(results.map(_.newPlan)), + allChildStagesReady = results.forall(_.allChildStagesReady)) + } + } + + private def createQueryStage(e: Exchange): QueryStage = { + val optimizedPlan = postStageCreationOptimize(e.child) + val queryStage = e match { + case s: ShuffleExchangeExec => + ShuffleQueryStage(currentStageId, s.copy(child = optimizedPlan)) + case b: BroadcastExchangeExec => + BroadcastQueryStage(currentStageId, b.copy(child = optimizedPlan)) + } + currentStageId += 1 + post(MaterializeStage(queryStage)) + queryStage + } + + override protected def onError(e: Throwable): Unit = callback.onError(e) +} + +case class CreateStageResult(newPlan: SparkPlan, allChildStagesReady: Boolean) + +object QueryStageCreator { + private val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("QueryStageCreator", 16)) +} + +trait QueryStageTriggerCallback { + def onPlanUpdate(updatedPlan: SparkPlan): Unit + def onStageMaterializingFailed(stage: QueryStage, e: Throwable): Unit + def onError(e: Throwable): Unit +} + +sealed trait QueryStageCreatorEvent + +object StartCreation extends QueryStageCreatorEvent + +case class MaterializeStage(stage: QueryStage) extends QueryStageCreatorEvent + +case class StageReady(stage: QueryStage) extends QueryStageCreatorEvent diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageTrigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageTrigger.scala deleted file mode 100644 index 5dd65f42442b2..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageTrigger.scala +++ /dev/null @@ -1,160 +0,0 @@ -/* - * 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.{HashMap, HashSet, ListBuffer} -import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService, Future} - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.util.{EventLoop, ThreadUtils} - -/** - * This class triggers [[QueryStage]] bottom-up, apply planner rules for query stages and - * materialize them. It triggers as many query stages as possible at the same time, and triggers - * the parent query stage when all its child stages are materialized. - */ -class QueryStageTrigger(session: SparkSession, callback: QueryStageTriggerCallback) - extends EventLoop[QueryStageTriggerEvent]("QueryStageTrigger") { - - private val stageToParentStages = HashMap.empty[Int, ListBuffer[QueryStage]] - - private val idToUpdatedStage = HashMap.empty[Int, QueryStage] - - private val stageToNumPendingChildStages = HashMap.empty[Int, Int] - - private val submittedStages = HashSet.empty[Int] - - private val readyStages = HashSet.empty[Int] - - private val planner = new QueryStagePlanner(session.sessionState.conf) - - def trigger(stage: QueryStage): Unit = { - post(SubmitStage(stage)) - } - - private implicit def executionContext: ExecutionContextExecutorService = { - QueryStageTrigger.executionContext - } - - override protected def onReceive(event: QueryStageTriggerEvent): Unit = event match { - case SubmitStage(stage) => - // We may submit a query stage multiple times, because of stage reuse. Here we avoid - // re-submitting a query stage. - if (!submittedStages.contains(stage.id)) { - submittedStages += stage.id - val pendingChildStages = stage.plan.collect { - // The stage being submitted may have child stages that are already ready, if the child - // stage is a reused stage. - case stage: QueryStage if !readyStages.contains(stage.id) => stage - } - if (pendingChildStages.isEmpty) { - // This is a leaf stage, or all its child stages are ready, we can plan it now. - post(PlanStage(stage)) - } else { - // This stage has some pending child stages, we store the connection of this stage and - // its child stages, and submit all the child stages, so that we can plan this stage - // later when all its child stages are ready. - stageToNumPendingChildStages(stage.id) = pendingChildStages.length - pendingChildStages.foreach { child => - // a child may have multiple parents, because of query stage reuse. - val parentStages = stageToParentStages.getOrElseUpdate(child.id, new ListBuffer) - parentStages += stage - post(SubmitStage(child)) - } - } - } - - case PlanStage(stage) => - Future { - // planning needs active SparkSession in current thread. - SparkSession.setActiveSession(session) - planner.execute(stage.plan) - }.onComplete { res => - if (res.isSuccess) { - post(StagePlanned(stage, res.get)) - } else { - callback.onStagePlanningFailed(stage, res.failed.get) - stop() - } - } - submittedStages += stage.id - - case StagePlanned(stage, optimizedPlan) => - val newStage = stage.withNewPlan(optimizedPlan) - // We store the new stage with the new query plan after planning, so that later on we can - // update the query plan of its parent stage. - idToUpdatedStage(newStage.id) = newStage - // This stage has optimized its plan, notify the callback about this change. - callback.onStageUpdated(newStage) - - newStage.materialize().onComplete { res => - if (res.isSuccess) { - post(StageReady(stage)) - } else { - callback.onStageMaterializingFailed(newStage, res.failed.get) - stop() - } - } - - case StageReady(stage) => - readyStages += stage.id - stageToParentStages.remove(stage.id).foreach { parentStages => - parentStages.foreach { parent => - val numPendingChildStages = stageToNumPendingChildStages(parent.id) - if (numPendingChildStages == 1) { - stageToNumPendingChildStages.remove(parent.id) - // All its child stages are ready, here we update the query plan via replacing the old - // child stages with new ones that are planned. - val newPlan = parent.plan.transform { - case q: QueryStage => idToUpdatedStage(q.id) - } - // We can plan this stage now. - post(PlanStage(parent.withNewPlan(newPlan))) - } else { - assert(numPendingChildStages > 1) - stageToNumPendingChildStages(parent.id) = numPendingChildStages - 1 - } - } - } - } - - override protected def onError(e: Throwable): Unit = callback.onError(e) -} - -object QueryStageTrigger { - private val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("QueryStageTrigger", 16)) -} - -trait QueryStageTriggerCallback { - def onStageUpdated(stage: QueryStage): Unit - def onStagePlanningFailed(stage: QueryStage, e: Throwable): Unit - def onStageMaterializingFailed(stage: QueryStage, e: Throwable): Unit - def onError(e: Throwable): Unit -} - -sealed trait QueryStageTriggerEvent - -case class SubmitStage(stage: QueryStage) extends QueryStageTriggerEvent - -case class PlanStage(stage: QueryStage) extends QueryStageTriggerEvent - -case class StagePlanned(stage: QueryStage, optimizedPlan: SparkPlan) extends QueryStageTriggerEvent - -case class StageReady(stage: QueryStage) extends QueryStageTriggerEvent diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStagePlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala similarity index 62% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStagePlanner.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala index 0319d146c30dc..33cd6f74ad29e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStagePlanner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala @@ -15,25 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.adaptive +package org.apache.spark.sql.execution.adaptive.rule -import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} -import org.apache.spark.sql.execution.{CollapseCodegenStages, SparkPlan} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.QueryStage -class QueryStagePlanner(conf: SQLConf) extends RuleExecutor[SparkPlan] { - - override protected def batches: Seq[Batch] = Seq( - Batch("QueryStage Optimization", Once, - AssertChildStagesMaterialized, - ReduceNumShufflePartitions(conf), - CollapseCodegenStages(conf) - ) - ) -} - -// A sanity check rule to make sure we are running `QueryStagePlanner` on a sub-tree of query plan -// with all input stages materialized. +// A sanity check rule to make sure we are running query stage optimizer rules on a sub-tree of +// query plan with all input stages materialized. object AssertChildStagesMaterialized extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = plan.transform { case q: QueryStage if !q.materialize().isCompleted => 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/rule/ReduceNumShufflePartitions.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala index 1f3e85c153dad..9849ef9f4503b 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/rule/ReduceNumShufflePartitions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.adaptive +package org.apache.spark.sql.execution.adaptive.rule import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration.Duration @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.adaptive.ShuffleQueryStage import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ThreadUtils 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 70a7ed0e8d1d1..00bcee330afde 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 @@ -23,6 +23,7 @@ import org.apache.spark.{MapOutputStatistics, SparkConf, SparkFunSuite} import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.adaptive.rule.{CoalescedShuffleReaderExec, ReduceNumShufflePartitions} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -314,7 +315,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = agg.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -361,7 +362,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -413,7 +414,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -465,7 +466,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -508,7 +509,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's make sure we do not reduce number of ppst shuffle partitions. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -533,7 +534,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // ReusedQueryStage 0 val resultDf = df.join(df, "key").join(df, "key") val finalPlan = resultDf.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].resultStage.plan + .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan assert(finalPlan.collect { case p: ReusedQueryStage => p }.length == 2) assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 3) checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) @@ -549,7 +550,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA .union(grouped.groupBy(col("key") + 2).max("value")) val resultStage = resultDf2.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].resultStage + .asInstanceOf[AdaptiveSparkPlan].finalPlan // The result stage has 2 children val level1Stages = resultStage.plan.collect { case q: QueryStage => q } From 4e69702f01b22f5f07038bfd0fd28020b3e69a4f Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Fri, 1 Feb 2019 17:53:12 +0800 Subject: [PATCH 13/21] address comments --- .../sql/execution/adaptive/AdaptiveSparkPlan.scala | 4 ++-- .../sql/execution/adaptive/QueryStageCreator.scala | 9 ++++++--- .../adaptive/rule/AssertChildStagesMaterialized.scala | 2 +- .../adaptive/rule/ReduceNumShufflePartitions.scala | 10 ++++------ 4 files changed, 13 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala index 44f6c6f497918..32f678a8fe680 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala @@ -46,8 +46,8 @@ case class AdaptiveSparkPlan(initialPlan: SparkPlan, session: SparkSession) // by calling `readyLock.getCount()`. private val readyLock = new CountDownLatch(1) - private def createCallback(executionId: Option[Long]): QueryStageTriggerCallback = { - new QueryStageTriggerCallback { + private def createCallback(executionId: Option[Long]): QueryStageCreatorCallback = { + new QueryStageCreatorCallback { override def onPlanUpdate(updatedPlan: SparkPlan): Unit = { updateCurrentPlan(updatedPlan, executionId) if (updatedPlan.isInstanceOf[ResultQueryStage]) readyLock.countDown() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala index 56f883f5123f8..98c06d3beaa68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala @@ -61,7 +61,7 @@ import org.apache.spark.util.{EventLoop, ThreadUtils} class QueryStageCreator( initialPlan: SparkPlan, session: SparkSession, - callback: QueryStageTriggerCallback) + callback: QueryStageCreatorCallback) extends EventLoop[QueryStageCreatorEvent]("QueryStageCreator") { private def conf = session.sessionState.conf @@ -90,14 +90,17 @@ class QueryStageCreator( private var currentPlan = initialPlan + private val localProperties = session.sparkContext.getLocalProperties + private implicit def executionContext: ExecutionContextExecutorService = { QueryStageCreator.executionContext } override protected def onReceive(event: QueryStageCreatorEvent): Unit = event match { case StartCreation => - // set active session for the event loop thread. + // set active session and local properties for the event loop thread. SparkSession.setActiveSession(session) + session.sparkContext.setLocalProperties(localProperties) currentPlan = createQueryStages(initialPlan) case MaterializeStage(stage) => @@ -240,7 +243,7 @@ object QueryStageCreator { ThreadUtils.newDaemonCachedThreadPool("QueryStageCreator", 16)) } -trait QueryStageTriggerCallback { +trait QueryStageCreatorCallback { def onPlanUpdate(updatedPlan: SparkPlan): Unit def onStageMaterializingFailed(stage: QueryStage, e: Throwable): Unit def onError(e: Throwable): Unit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala index 33cd6f74ad29e..0afa8397e3312 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala @@ -27,6 +27,6 @@ object AssertChildStagesMaterialized extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = plan.transform { case q: QueryStage if !q.materialize().isCompleted => throw new IllegalArgumentException( - s"The input stages should all be materialize, but ${q.id} is not.") + s"The input stages should all be materialized, but the below one is not.\n ${q.plan}") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala index 9849ef9f4503b..c46607c8efb3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.adaptive.ShuffleQueryStage +import org.apache.spark.sql.execution.adaptive.{QueryStage, ShuffleQueryStage} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ThreadUtils @@ -61,11 +61,9 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { ThreadUtils.awaitResult(metricsFuture, Duration.Zero) } - val leafNodes = plan.collect { - case s: SparkPlan if s.children.isEmpty => s - } + val allStageLeaves = plan.collectLeaves().forall(_.isInstanceOf[QueryStage]) - if (shuffleMetrics.length == leafNodes.length) { + if (allStageLeaves) { // ShuffleQueryStage gives null mapOutputStatistics when the input RDD has 0 partitions, // we should skip it when calculating the `partitionStartIndices`. val validMetrics = shuffleMetrics.filter(_ != null) @@ -83,7 +81,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { plan } } else { - // If not all leaf nodes are shuffle query stages, it's not safe to reduce the number of + // 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. plan From 41f3a90175d6c6ab2e760d371d0ea75f1af4a7d7 Mon Sep 17 00:00:00 2001 From: Li Yuanjian Date: Thu, 14 Feb 2019 12:59:26 +0800 Subject: [PATCH 14/21] Fix UT (#7) --- .../spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index 4a1297b71feb8..adbf9e9deb238 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -34,5 +34,6 @@ case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan case _: ExecutedCommandExec => plan case _ if session.sessionState.conf.adaptiveExecutionEnabled => AdaptiveSparkPlan(plan, session.cloneSession()) + case _ => plan } } From 666bf76362eb9d2769b6bfc9b18328f9f74b7921 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Mon, 25 Feb 2019 16:45:41 +0800 Subject: [PATCH 15/21] rename to QueryFragment --- .../spark/sql/execution/SparkPlanInfo.scala | 6 +- ...Plan.scala => AdaptiveSparkPlanExec.scala} | 30 +- .../adaptive/InsertAdaptiveSparkPlan.scala | 4 +- .../adaptive/QueryFragmentCreator.scala | 260 ++++++++++++++++++ ...eryStage.scala => QueryFragmentExec.scala} | 50 ++-- .../adaptive/QueryStageCreator.scala | 258 ----------------- ...=> AssertChildFragmentsMaterialized.scala} | 8 +- .../rule/ReduceNumShufflePartitions.scala | 32 +-- .../sql/execution/ui/SparkPlanGraph.scala | 2 +- .../ReduceNumShufflePartitionsSuite.scala | 60 ++-- 10 files changed, 358 insertions(+), 352 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/{AdaptiveSparkPlan.scala => AdaptiveSparkPlanExec.scala} (77%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentCreator.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/{QueryStage.scala => QueryFragmentExec.scala} (69%) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/{AssertChildStagesMaterialized.scala => AssertChildFragmentsMaterialized.scala} (80%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 9c6d5928259d5..fef74ea7cd3f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlan, QueryStage} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryFragmentExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo import org.apache.spark.sql.internal.SQLConf @@ -53,8 +53,8 @@ private[execution] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val children = plan match { case ReusedExchangeExec(_, child) => child :: Nil - case a: AdaptiveSparkPlan => a.finalPlan.plan :: Nil - case stage: QueryStage => stage.plan :: Nil + case a: AdaptiveSparkPlanExec => a.finalPlan.plan :: Nil + case stage: QueryFragmentExec => stage.plan :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala similarity index 77% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 32f678a8fe680..7da258842286b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -28,11 +28,11 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, SparkPlanInfo, S import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate /** - * A root node to execute the query plan adaptively. It creates query stages, and incrementally - * updates the query plan when a query stage is materialized and provides accurate runtime + * A root node to execute the query plan adaptively. It creates query fragments, and incrementally + * updates the query plan when a query fragment is materialized and provides accurate runtime * data statistics. */ -case class AdaptiveSparkPlan(initialPlan: SparkPlan, session: SparkSession) +case class AdaptiveSparkPlanExec(initialPlan: SparkPlan, session: SparkSession) extends LeafExecNode{ override def output: Seq[Attribute] = initialPlan.output @@ -40,24 +40,26 @@ case class AdaptiveSparkPlan(initialPlan: SparkPlan, session: SparkSession) @volatile private var currentPlan: SparkPlan = initialPlan @volatile private var error: Throwable = null - // We will release the lock when we finish planning query stages, or we fail to do the planning. - // Getting `resultStage` will be blocked until the lock is release. + // We will release the lock when we finish planning query fragments, or we fail to do the + // planning. Getting `finalPlan` will be blocked until the lock is release. // This is better than wait()/notify(), as we can easily check if the computation has completed, // by calling `readyLock.getCount()`. private val readyLock = new CountDownLatch(1) - private def createCallback(executionId: Option[Long]): QueryStageCreatorCallback = { - new QueryStageCreatorCallback { + private def createCallback(executionId: Option[Long]): QueryFragmentCreatorCallback = { + new QueryFragmentCreatorCallback { override def onPlanUpdate(updatedPlan: SparkPlan): Unit = { updateCurrentPlan(updatedPlan, executionId) - if (updatedPlan.isInstanceOf[ResultQueryStage]) readyLock.countDown() + if (updatedPlan.isInstanceOf[ResultQueryFragmentExec]) readyLock.countDown() } - override def onStageMaterializingFailed(stage: QueryStage, e: Throwable): Unit = { + override def onFragmentMaterializingFailed( + fragment: QueryFragmentExec, + e: Throwable): Unit = { error = new SparkException( s""" - |Fail to materialize stage ${stage.id}: - |${stage.plan.treeString} + |Fail to materialize fragment ${fragment.id}: + |${fragment.plan.treeString} """.stripMargin, e) readyLock.countDown() } @@ -79,18 +81,18 @@ case class AdaptiveSparkPlan(initialPlan: SparkPlan, session: SparkSession) } } - def finalPlan: ResultQueryStage = { + def finalPlan: ResultQueryFragmentExec = { if (readyLock.getCount > 0) { val sc = session.sparkContext val executionId = Option(sc.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).map(_.toLong) - val creator = new QueryStageCreator(initialPlan, session, createCallback(executionId)) + val creator = new QueryFragmentCreator(initialPlan, session, createCallback(executionId)) creator.start() readyLock.await() creator.stop() } if (error != null) throw error - currentPlan.asInstanceOf[ResultQueryStage] + currentPlan.asInstanceOf[ResultQueryFragmentExec] } override def executeCollect(): Array[InternalRow] = finalPlan.executeCollect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index adbf9e9deb238..5ab23962f4c6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.ExecutedCommandExec /** - * This rule wraps the query plan with an [[AdaptiveSparkPlan]], which executes the query plan + * This rule wraps the query plan with an [[AdaptiveSparkPlanExec]], which executes the query plan * adaptively with runtime data statistics. Note that this rule must be run after * [[org.apache.spark.sql.execution.exchange.EnsureRequirements]], so that the exchange nodes are * already inserted. @@ -33,7 +33,7 @@ case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan override def apply(plan: SparkPlan): SparkPlan = plan match { case _: ExecutedCommandExec => plan case _ if session.sessionState.conf.adaptiveExecutionEnabled => - AdaptiveSparkPlan(plan, session.cloneSession()) + AdaptiveSparkPlanExec(plan, session.cloneSession()) case _ => plan } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentCreator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentCreator.scala new file mode 100644 index 0000000000000..49f7a271872b3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentCreator.scala @@ -0,0 +1,260 @@ +/* + * 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.{ExecutionContext, ExecutionContextExecutorService} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{CollapseCodegenStages, SparkPlan} +import org.apache.spark.sql.execution.adaptive.rule.{AssertChildFragmentsMaterialized, ReduceNumShufflePartitions} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ShuffleExchangeExec} +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.{EventLoop, ThreadUtils} + +/** + * This class dynamically creates [[QueryFragmentExec]] bottom-up, optimize the query plan of query + * fragments and materialize them. It creates as many query fragments as possible at the same time, + * and materialize a query fragment when all its child fragments are materialized. + * + * To create query fragments, we traverse the query tree bottom up. When we hit an exchange node, + * and all the child query fragments of this exchange node are materialized, we try to create a new + * query fragment for this exchange node. + * + * To create a new query fragment, we first optimize the sub-tree of the exchange. After + * optimization, we check the output partitioning of the optimized sub-tree, and see if the + * exchange node is still necessary. + * + * If the exchange node becomes unnecessary, remove it and give up this query fragment creation, + * and continue to traverse the query plan tree until we hit the next exchange node. + * + * If the exchange node is still needed, create the query fragment and optimize its sub-tree again. + * It's necessary to have both the pre-creation optimization and post-creation optimization, because + * these 2 optimization have different assumptions. For pre-creation optimization, the shuffle node + * may be removed later on and the current sub-tree may be only a part of a query fragment, so we + * don't have the big picture of the query fragment yet. For post-creation optimization, the query + * fragment is created and we have the big picture of the query fragment. + * + * After the query fragment is optimized, we materialize it asynchronously, and continue to traverse + * the query plan tree to create more query fragments. + * + * When a query fragment completes materialization, we trigger the process of query fragments + * creation and traverse the query plan tree again. + */ +class QueryFragmentCreator( + initialPlan: SparkPlan, + session: SparkSession, + callback: QueryFragmentCreatorCallback) + extends EventLoop[QueryFragmentCreatorEvent]("QueryFragmentCreator") { + + private def conf = session.sessionState.conf + + private val readyFragments = mutable.HashSet.empty[Int] + + private var currentFragmentId = 0 + + private val fragmentCache = + mutable.HashMap.empty[StructType, mutable.Buffer[(Exchange, QueryFragmentExec)]] + + // The optimizer rules that will be applied to a sub-tree of the query plan before the fragment is + // created. Note that we may end up not creating the query fragment, so the rules here should not + // assume the given sub-plan-tree is the entire query plan of the query fragment. For example, if + // a rule want to collect all the child query fragments, it should not be put here. + private val preFragmentCreationOptimizerRules: Seq[Rule[SparkPlan]] = Seq( + AssertChildFragmentsMaterialized + ) + + // The optimizer rules that will be applied to a sub-tree of the query plan after the fragment is + // created. Note that once the fragment is created, we will not remove it anymore. If a rule + // changes the output partitioning of the sub-plan-tree, which may help to remove the exchange + // node, it's better to put it in `preFragmentCreationOptimizerRules`, so that we may create less + // query fragments. + private val postFragmentCreationOptimizerRules: Seq[Rule[SparkPlan]] = Seq( + ReduceNumShufflePartitions(conf), + CollapseCodegenStages(conf)) + + private var currentPlan = initialPlan + + private val localProperties = session.sparkContext.getLocalProperties + + private implicit def executionContext: ExecutionContextExecutorService = { + QueryFragmentCreator.executionContext + } + + override protected def onReceive(event: QueryFragmentCreatorEvent): Unit = event match { + case StartCreation => + // set active session and local properties for the event loop thread. + SparkSession.setActiveSession(session) + session.sparkContext.setLocalProperties(localProperties) + currentPlan = createQueryFragments(initialPlan) + + case MaterializeFragment(fragment) => + fragment.materialize().onComplete { res => + if (res.isSuccess) { + post(FragmentReady(fragment)) + } else { + callback.onFragmentMaterializingFailed(fragment, res.failed.get) + stop() + } + } + + case FragmentReady(fragment) => + if (fragment.isInstanceOf[ResultQueryFragmentExec]) { + callback.onPlanUpdate(fragment) + stop() + } else { + readyFragments += fragment.id + currentPlan = createQueryFragments(currentPlan) + } + } + + override protected def onStart(): Unit = { + post(StartCreation) + } + + private def preFragmentCreationOptimize(plan: SparkPlan): SparkPlan = { + preFragmentCreationOptimizerRules.foldLeft(plan) { + case (current, rule) => rule(current) + } + } + + private def postFragmentCreationOptimize(plan: SparkPlan): SparkPlan = { + postFragmentCreationOptimizerRules.foldLeft(plan) { + case (current, rule) => rule(current) + } + } + + /** + * Traverse the query plan bottom-up, and creates query fragments as many as possible. + */ + private def createQueryFragments(plan: SparkPlan): SparkPlan = { + val result = createQueryFragments0(plan) + if (result.allChildFragmentsReady) { + val finalPlan = postFragmentCreationOptimize(preFragmentCreationOptimize(result.newPlan)) + post(FragmentReady(ResultQueryFragmentExec(currentFragmentId, finalPlan))) + finalPlan + } else { + callback.onPlanUpdate(result.newPlan) + result.newPlan + } + } + + /** + * This method is called recursively to traverse the plan tree bottom-up. This method returns two + * information: 1) the new plan after we insert query fragments. 2) whether or not the child query + * fragments of the new plan are all ready. + * + * if the current plan is an exchange node, and all its child query fragments are ready, we try to + * create a new query fragment. + */ + private def createQueryFragments0(plan: SparkPlan): CreateFragmentResult = plan match { + case e: Exchange => + val similarFragments = fragmentCache.getOrElseUpdate(e.schema, mutable.Buffer.empty) + similarFragments.find(_._1.sameResult(e)) match { + case Some((_, existingFragment)) if conf.exchangeReuseEnabled => + CreateFragmentResult( + newPlan = ReusedQueryFragmentExec(existingFragment, e.output), + allChildFragmentsReady = readyFragments.contains(existingFragment.id)) + + case _ => + val result = createQueryFragments0(e.child) + // Try to create a query fragment only when all the child query fragments are ready. + if (result.allChildFragmentsReady) { + val optimizedPlan = preFragmentCreationOptimize(result.newPlan) + e match { + case s: ShuffleExchangeExec => + (s.desiredPartitioning, optimizedPlan.outputPartitioning) match { + case (desired: HashPartitioning, actual: HashPartitioning) + if desired.semanticEquals(actual) => + // This shuffle exchange is unnecessary now, remove it. The reason maybe: + // 1. the child plan has changed its output partitioning after optimization, + // and makes this exchange node unnecessary. + // 2. this exchange node is user specified, which turns out to be unnecessary. + CreateFragmentResult(newPlan = optimizedPlan, allChildFragmentsReady = true) + case _ => + val queryFragment = createQueryFragment(s.copy(child = optimizedPlan)) + similarFragments.append(e -> queryFragment) + // We've created a new fragment, which is obviously not ready yet. + CreateFragmentResult(newPlan = queryFragment, allChildFragmentsReady = false) + } + + case b: BroadcastExchangeExec => + val queryFragment = createQueryFragment(b.copy(child = optimizedPlan)) + similarFragments.append(e -> queryFragment) + // We've created a new fragment, which is obviously not ready yet. + CreateFragmentResult(newPlan = queryFragment, allChildFragmentsReady = false) + } + } else { + CreateFragmentResult( + newPlan = e.withNewChildren(Seq(result.newPlan)), + allChildFragmentsReady = false) + } + } + + case q: QueryFragmentExec => + CreateFragmentResult(newPlan = q, allChildFragmentsReady = readyFragments.contains(q.id)) + + case _ => + if (plan.children.isEmpty) { + CreateFragmentResult(newPlan = plan, allChildFragmentsReady = true) + } else { + val results = plan.children.map(createQueryFragments0) + CreateFragmentResult( + newPlan = plan.withNewChildren(results.map(_.newPlan)), + allChildFragmentsReady = results.forall(_.allChildFragmentsReady)) + } + } + + private def createQueryFragment(e: Exchange): QueryFragmentExec = { + val optimizedPlan = postFragmentCreationOptimize(e.child) + val queryFragment = e match { + case s: ShuffleExchangeExec => + ShuffleQueryFragmentExec(currentFragmentId, s.copy(child = optimizedPlan)) + case b: BroadcastExchangeExec => + BroadcastQueryFragmentExec(currentFragmentId, b.copy(child = optimizedPlan)) + } + currentFragmentId += 1 + post(MaterializeFragment(queryFragment)) + queryFragment + } + + override protected def onError(e: Throwable): Unit = callback.onError(e) +} + +case class CreateFragmentResult(newPlan: SparkPlan, allChildFragmentsReady: Boolean) + +object QueryFragmentCreator { + private val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("QueryFragmentCreator", 16)) +} + +trait QueryFragmentCreatorCallback { + def onPlanUpdate(updatedPlan: SparkPlan): Unit + def onFragmentMaterializingFailed(fragment: QueryFragmentExec, e: Throwable): Unit + def onError(e: Throwable): Unit +} + +sealed trait QueryFragmentCreatorEvent + +object StartCreation extends QueryFragmentCreatorEvent + +case class MaterializeFragment(fragment: QueryFragmentExec) extends QueryFragmentCreatorEvent + +case class FragmentReady(fragment: QueryFragmentExec) extends QueryFragmentCreatorEvent diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala similarity index 69% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala index 6edd1e4eafb6f..bbc8722ea6590 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala @@ -29,32 +29,32 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange._ /** - * A query stage is an individual sub-tree of a query plan, which can be executed ahead and provide - * accurate data statistics. For example, a sub-tree under shuffle/broadcast node is a query stage. - * Each query stage runs in a single Spark job/stage. + * A query fragment is an individual sub-tree of a query plan, which can be executed ahead and + * provide accurate data statistics. For example, a sub-tree under shuffle/broadcast node is a + * query fragment. Each query fragment runs in a single Spark job/stage. */ -abstract class QueryStage extends LeafExecNode { +abstract class QueryFragmentExec extends LeafExecNode { /** - * An id of this query stage which is unique in the entire query plan. + * An id of this query fragment which is unique in the entire query plan. */ def id: Int /** - * The sub-tree of the query plan that belongs to this query stage. + * The sub-tree of the query plan that belongs to this query fragment. */ def plan: SparkPlan /** - * Returns a new query stage with a new plan, which is optimized based on accurate runtime + * Returns a new query fragment with a new plan, which is optimized based on accurate runtime * statistics. */ - def withNewPlan(newPlan: SparkPlan): QueryStage + def withNewPlan(newPlan: SparkPlan): QueryFragmentExec /** - * Materialize this QueryStage, to prepare for the execution, like submitting map stages, + * Materialize this QueryFragment, to prepare for the execution, like submitting map stages, * broadcasting data, etc. The caller side can use the returned [[Future]] to wait until this - * stage is ready. + * fragment is ready. */ def materialize(): Future[Any] @@ -68,7 +68,7 @@ abstract class QueryStage extends LeafExecNode { override def doExecuteBroadcast[T](): Broadcast[T] = plan.executeBroadcast() override def doCanonicalize(): SparkPlan = plan.canonicalized - // TODO: maybe we should not hide QueryStage entirely from explain result. + // TODO: maybe we should not hide QueryFragment entirely from explain result. override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], @@ -83,25 +83,25 @@ abstract class QueryStage extends LeafExecNode { } /** - * The last QueryStage of an execution plan. + * The last QueryFragment of an execution plan. */ -case class ResultQueryStage(id: Int, plan: SparkPlan) extends QueryStage { +case class ResultQueryFragmentExec(id: Int, plan: SparkPlan) extends QueryFragmentExec { override def materialize(): Future[Any] = { - throw new IllegalStateException("Cannot materialize ResultQueryStage.") + throw new IllegalStateException("Cannot materialize ResultQueryFragment.") } - override def withNewPlan(newPlan: SparkPlan): QueryStage = { + override def withNewPlan(newPlan: SparkPlan): QueryFragmentExec = { copy(plan = newPlan) } } /** - * A shuffle QueryStage whose child is a [[ShuffleExchangeExec]]. + * A shuffle QueryFragment whose child is a [[ShuffleExchangeExec]]. */ -case class ShuffleQueryStage(id: Int, plan: ShuffleExchangeExec) extends QueryStage { +case class ShuffleQueryFragmentExec(id: Int, plan: ShuffleExchangeExec) extends QueryFragmentExec { - override def withNewPlan(newPlan: SparkPlan): QueryStage = { + override def withNewPlan(newPlan: SparkPlan): QueryFragmentExec = { copy(plan = newPlan.asInstanceOf[ShuffleExchangeExec]) } @@ -121,11 +121,12 @@ case class ShuffleQueryStage(id: Int, plan: ShuffleExchangeExec) extends QuerySt } /** - * A broadcast QueryStage whose child is a [[BroadcastExchangeExec]]. + * A broadcast QueryFragment whose child is a [[BroadcastExchangeExec]]. */ -case class BroadcastQueryStage(id: Int, plan: BroadcastExchangeExec) extends QueryStage { +case class BroadcastQueryFragmentExec(id: Int, plan: BroadcastExchangeExec) + extends QueryFragmentExec { - override def withNewPlan(newPlan: SparkPlan): QueryStage = { + override def withNewPlan(newPlan: SparkPlan): QueryFragmentExec = { copy(plan = newPlan.asInstanceOf[BroadcastExchangeExec]) } @@ -135,9 +136,10 @@ case class BroadcastQueryStage(id: Int, plan: BroadcastExchangeExec) extends Que } /** - * A wrapper of QueryStage to indicate that it's reused. Note that this is not a query stage. + * A wrapper of QueryFragment to indicate that it's reused. Note that this is not a query fragment. */ -case class ReusedQueryStage(child: SparkPlan, output: Seq[Attribute]) extends UnaryExecNode { +case class ReusedQueryFragmentExec(child: QueryFragmentExec, output: Seq[Attribute]) + extends UnaryExecNode { // Ignore this wrapper for canonicalizing. override def doCanonicalize(): SparkPlan = child.canonicalized @@ -150,7 +152,7 @@ case class ReusedQueryStage(child: SparkPlan, output: Seq[Attribute]) extends Un child.executeBroadcast() } - // `ReusedQueryStage` can have distinct set of output attribute ids from its child, we need + // `ReusedQueryFragment` can have distinct set of output attribute ids from its child, we need // to update the attribute ids in `outputPartitioning` and `outputOrdering`. private lazy val updateAttr: Expression => Expression = { val originalAttrToNewAttr = AttributeMap(child.output.zip(output)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala deleted file mode 100644 index 98c06d3beaa68..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageCreator.scala +++ /dev/null @@ -1,258 +0,0 @@ -/* - * 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.{ExecutionContext, ExecutionContextExecutorService} - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{CollapseCodegenStages, SparkPlan} -import org.apache.spark.sql.execution.adaptive.rule.{AssertChildStagesMaterialized, ReduceNumShufflePartitions} -import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ShuffleExchangeExec} -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.{EventLoop, ThreadUtils} - -/** - * This class dynamically creates [[QueryStage]] bottom-up, optimize the query plan of query stages - * and materialize them. It creates as many query stages as possible at the same time, and - * materialize a query stage when all its child stages are materialized. - * - * To create query stages, we traverse the query tree bottom up. When we hit an exchange node, and - * all the child query stages of this exchange node are materialized, we try to create a new query - * stage for this exchange node. - * - * To create a new query stage, we first optimize the sub-tree of the exchange. After optimization, - * we check the output partitioning of the optimized sub-tree, and see if the exchange node is still - * necessary. - * - * If the exchange node becomes unnecessary, remove it and give up this query stage creation, and - * continue to traverse the query plan tree until we hit the next exchange node. - * - * If the exchange node is still needed, create the query stage and optimize its sub-tree again. - * It's necessary to have both the pre-creation optimization and post-creation optimization, because - * these 2 optimization have different assumptions. For pre-creation optimization, the shuffle node - * may be removed later on and the current sub-tree may be only a part of a query stage, so we don't - * have the big picture of the query stage yet. For post-creation optimization, the query stage is - * created and we have the big picture of the query stage. - * - * After the query stage is optimized, we materialize it asynchronously, and continue to traverse - * the query plan tree to create more query stages. - * - * When a query stage completes materialization, we trigger the process of query stages creation and - * traverse the query plan tree again. - */ -class QueryStageCreator( - initialPlan: SparkPlan, - session: SparkSession, - callback: QueryStageCreatorCallback) - extends EventLoop[QueryStageCreatorEvent]("QueryStageCreator") { - - private def conf = session.sessionState.conf - - private val readyStages = mutable.HashSet.empty[Int] - - private var currentStageId = 0 - - private val stageCache = mutable.HashMap.empty[StructType, mutable.Buffer[(Exchange, QueryStage)]] - - // The optimizer rules that will be applied to a sub-tree of the query plan before the stage is - // created. Note that we may end up not creating the query stage, so the rules here should not - // assume the given sub-plan-tree is the entire query plan of the query stage. For example, if a - // rule want to collect all the child query stages, it should not be put here. - private val preStageCreationOptimizerRules: Seq[Rule[SparkPlan]] = Seq( - AssertChildStagesMaterialized - ) - - // The optimizer rules that will be applied to a sub-tree of the query plan after the stage is - // created. Note that once the stage is created, we will not remove it anymore. If a rule changes - // the output partitioning of the sub-plan-tree, which may help to remove the exchange node, it's - // better to put it in `preStageCreationOptimizerRules`, so that we may create less query stages. - private val postStageCreationOptimizerRules: Seq[Rule[SparkPlan]] = Seq( - ReduceNumShufflePartitions(conf), - CollapseCodegenStages(conf)) - - private var currentPlan = initialPlan - - private val localProperties = session.sparkContext.getLocalProperties - - private implicit def executionContext: ExecutionContextExecutorService = { - QueryStageCreator.executionContext - } - - override protected def onReceive(event: QueryStageCreatorEvent): Unit = event match { - case StartCreation => - // set active session and local properties for the event loop thread. - SparkSession.setActiveSession(session) - session.sparkContext.setLocalProperties(localProperties) - currentPlan = createQueryStages(initialPlan) - - case MaterializeStage(stage) => - stage.materialize().onComplete { res => - if (res.isSuccess) { - post(StageReady(stage)) - } else { - callback.onStageMaterializingFailed(stage, res.failed.get) - stop() - } - } - - case StageReady(stage) => - if (stage.isInstanceOf[ResultQueryStage]) { - callback.onPlanUpdate(stage) - stop() - } else { - readyStages += stage.id - currentPlan = createQueryStages(currentPlan) - } - } - - override protected def onStart(): Unit = { - post(StartCreation) - } - - private def preStageCreationOptimize(plan: SparkPlan): SparkPlan = { - preStageCreationOptimizerRules.foldLeft(plan) { - case (current, rule) => rule(current) - } - } - - private def postStageCreationOptimize(plan: SparkPlan): SparkPlan = { - postStageCreationOptimizerRules.foldLeft(plan) { - case (current, rule) => rule(current) - } - } - - /** - * Traverse the query plan bottom-up, and creates query stages as many as possible. - */ - private def createQueryStages(plan: SparkPlan): SparkPlan = { - val result = createQueryStages0(plan) - if (result.allChildStagesReady) { - val finalPlan = postStageCreationOptimize(preStageCreationOptimize(result.newPlan)) - post(StageReady(ResultQueryStage(currentStageId, finalPlan))) - finalPlan - } else { - callback.onPlanUpdate(result.newPlan) - result.newPlan - } - } - - /** - * This method is called recursively to traverse the plan tree bottom-up. This method returns two - * information: 1) the new plan after we insert query stages. 2) whether or not the child query - * stages of the new plan are all ready. - * - * if the current plan is an exchange node, and all its child query stages are ready, we try to - * create a new query stage. - */ - private def createQueryStages0(plan: SparkPlan): CreateStageResult = plan match { - case e: Exchange => - val similarStages = stageCache.getOrElseUpdate(e.schema, mutable.Buffer.empty) - similarStages.find(_._1.sameResult(e)) match { - case Some((_, existingStage)) if conf.exchangeReuseEnabled => - CreateStageResult( - newPlan = ReusedQueryStage(existingStage, e.output), - allChildStagesReady = readyStages.contains(existingStage.id)) - - case _ => - val result = createQueryStages0(e.child) - // Try to create a query stage only when all the child query stages are ready. - if (result.allChildStagesReady) { - val optimizedPlan = preStageCreationOptimize(result.newPlan) - e match { - case s: ShuffleExchangeExec => - (s.desiredPartitioning, optimizedPlan.outputPartitioning) match { - case (desired: HashPartitioning, actual: HashPartitioning) - if desired.semanticEquals(actual) => - // This shuffle exchange is unnecessary now, remove it. The reason maybe: - // 1. the child plan has changed its output partitioning after optimization, - // and makes this exchange node unnecessary. - // 2. this exchange node is user specified, which turns out to be unnecessary. - CreateStageResult(newPlan = optimizedPlan, allChildStagesReady = true) - case _ => - val queryStage = createQueryStage(s.copy(child = optimizedPlan)) - similarStages.append(e -> queryStage) - // We've created a new stage, which is obviously not ready yet. - CreateStageResult(newPlan = queryStage, allChildStagesReady = false) - } - - case b: BroadcastExchangeExec => - val queryStage = createQueryStage(b.copy(child = optimizedPlan)) - similarStages.append(e -> queryStage) - // We've created a new stage, which is obviously not ready yet. - CreateStageResult(newPlan = queryStage, allChildStagesReady = false) - } - } else { - CreateStageResult( - newPlan = e.withNewChildren(Seq(result.newPlan)), - allChildStagesReady = false) - } - } - - case q: QueryStage => - CreateStageResult(newPlan = q, allChildStagesReady = readyStages.contains(q.id)) - - case _ => - if (plan.children.isEmpty) { - CreateStageResult(newPlan = plan, allChildStagesReady = true) - } else { - val results = plan.children.map(createQueryStages0) - CreateStageResult( - newPlan = plan.withNewChildren(results.map(_.newPlan)), - allChildStagesReady = results.forall(_.allChildStagesReady)) - } - } - - private def createQueryStage(e: Exchange): QueryStage = { - val optimizedPlan = postStageCreationOptimize(e.child) - val queryStage = e match { - case s: ShuffleExchangeExec => - ShuffleQueryStage(currentStageId, s.copy(child = optimizedPlan)) - case b: BroadcastExchangeExec => - BroadcastQueryStage(currentStageId, b.copy(child = optimizedPlan)) - } - currentStageId += 1 - post(MaterializeStage(queryStage)) - queryStage - } - - override protected def onError(e: Throwable): Unit = callback.onError(e) -} - -case class CreateStageResult(newPlan: SparkPlan, allChildStagesReady: Boolean) - -object QueryStageCreator { - private val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("QueryStageCreator", 16)) -} - -trait QueryStageCreatorCallback { - def onPlanUpdate(updatedPlan: SparkPlan): Unit - def onStageMaterializingFailed(stage: QueryStage, e: Throwable): Unit - def onError(e: Throwable): Unit -} - -sealed trait QueryStageCreatorEvent - -object StartCreation extends QueryStageCreatorEvent - -case class MaterializeStage(stage: QueryStage) extends QueryStageCreatorEvent - -case class StageReady(stage: QueryStage) extends QueryStageCreatorEvent diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala similarity index 80% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala index 0afa8397e3312..97ba33e0e3a18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala @@ -19,14 +19,14 @@ package org.apache.spark.sql.execution.adaptive.rule import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.adaptive.QueryStage +import org.apache.spark.sql.execution.adaptive.QueryFragmentExec // A sanity check rule to make sure we are running query stage optimizer rules on a sub-tree of // query plan with all input stages materialized. -object AssertChildStagesMaterialized extends Rule[SparkPlan] { +object AssertChildFragmentsMaterialized extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = plan.transform { - case q: QueryStage if !q.materialize().isCompleted => + case q: QueryFragmentExec if !q.materialize().isCompleted => throw new IllegalArgumentException( - s"The input stages should all be materialized, but the below one is not.\n ${q.plan}") + s"The input fragments should all be materialized, but the below one is not.\n ${q.plan}") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala index c46607c8efb3e..8ad75a0c7adc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.adaptive.{QueryStage, ShuffleQueryStage} +import org.apache.spark.sql.execution.adaptive.{QueryFragmentExec, ShuffleQueryFragmentExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ThreadUtils @@ -55,16 +55,16 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { val shuffleMetrics: Seq[MapOutputStatistics] = plan.collect { - case stage: ShuffleQueryStage => - val metricsFuture = stage.mapOutputStatisticsFuture - assert(metricsFuture.isCompleted, "ShuffleQueryStage should already be ready") + case fragment: ShuffleQueryFragmentExec => + val metricsFuture = fragment.mapOutputStatisticsFuture + assert(metricsFuture.isCompleted, "ShuffleQueryFragment should already be ready") ThreadUtils.awaitResult(metricsFuture, Duration.Zero) } - val allStageLeaves = plan.collectLeaves().forall(_.isInstanceOf[QueryStage]) + val allFragmentLeaves = plan.collectLeaves().forall(_.isInstanceOf[QueryFragmentExec]) - if (allStageLeaves) { - // ShuffleQueryStage gives null mapOutputStatistics when the input RDD has 0 partitions, + if (allFragmentLeaves) { + // ShuffleQueryFragment gives null mapOutputStatistics when the input RDD has 0 partitions, // we should skip it when calculating the `partitionStartIndices`. val validMetrics = shuffleMetrics.filter(_ != null) if (validMetrics.nonEmpty) { @@ -72,16 +72,16 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // 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: ShuffleQueryStage => - CoalescedShuffleReaderExec(stage, partitionStartIndices) + // `partitionStartIndices`, so that all the leaf shuffles in a fragment have the same + // number of output partitions. + case fragment: ShuffleQueryFragmentExec => + CoalescedShuffleReaderExec(fragment, partitionStartIndices) } } else { plan } } else { - // If not all leaf nodes are query stages, it's not safe to reduce the number of + // If not all leaf nodes are query fragments, 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. plan @@ -90,7 +90,7 @@ 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. + * mapOutputStatistics provided by all pre-shuffle fragments. */ // visible for testing. private[sql] def estimatePartitionStartIndices( @@ -114,7 +114,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " + s"targetPostShuffleInputSize $targetPostShuffleInputSize.") - // Make sure we do get the same number of pre-shuffle partitions for those stages. + // Make sure we do get the same number of pre-shuffle partitions for those fragments. val distinctNumPreShufflePartitions = mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct // The reason that we are expecting a single value of the number of pre-shuffle partitions @@ -137,7 +137,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { var i = 0 while (i < numPreShufflePartitions) { - // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages. + // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle fragments. // Then, we add the total size to postShuffleInputSize. var nextShuffleInputSize = 0L var j = 0 @@ -162,7 +162,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { } case class CoalescedShuffleReaderExec( - child: ShuffleQueryStage, + child: ShuffleQueryFragmentExec, partitionStartIndices: Array[Int]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index 113b205367a59..c5453289d3946 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -96,7 +96,7 @@ object SparkPlanGraph { case "InputAdapter" => buildSparkPlanGraphNode( planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) - case "BroadcastQueryStage" | "ResultQueryStage" | "ShuffleQueryStage" => + case "BroadcastQueryFragment" | "ResultQueryFragment" | "ShuffleQueryFragment" => if (exchanges.contains(planInfo.children.head)) { // Point to the re-used exchange val node = exchanges(planInfo.children.head) 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 00bcee330afde..4e268663569cd 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 @@ -315,7 +315,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = agg.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -362,7 +362,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -414,7 +414,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -466,7 +466,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -509,7 +509,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's make sure we do not reduce number of ppst shuffle partitions. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -527,47 +527,47 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA spark.sql("SET spark.sql.exchange.reuse=true") val df = spark.range(1).selectExpr("id AS key", "id AS value") - // test case 1: a stage has 3 child stages but they are the same stage. - // ResultQueryStage 1 - // ShuffleQueryStage 0 - // ReusedQueryStage 0 - // ReusedQueryStage 0 + // test case 1: a fragment has 3 child fragments but they are the same fragment. + // ResultQueryFragment 1 + // ShuffleQueryFragment 0 + // ReusedQueryFragment 0 + // ReusedQueryFragment 0 val resultDf = df.join(df, "key").join(df, "key") val finalPlan = resultDf.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].finalPlan.plan - assert(finalPlan.collect { case p: ReusedQueryStage => p }.length == 2) + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan + assert(finalPlan.collect { case p: ReusedQueryFragmentExec => p }.length == 2) assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 3) checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) - // test case 2: a stage has 2 parent stages. - // ResultQueryStage 3 - // ShuffleQueryStage 1 - // ShuffleQueryStage 0 - // ShuffleQueryStage 2 - // ReusedQueryStage 0 + // test case 2: a fragment has 2 parent fragments. + // ResultQueryFragment 3 + // ShuffleQueryFragment 1 + // ShuffleQueryFragment 0 + // ShuffleQueryFragment 2 + // ReusedQueryFragment 0 val grouped = df.groupBy("key").agg(max("value").as("value")) val resultDf2 = grouped.groupBy(col("key") + 1).max("value") .union(grouped.groupBy(col("key") + 2).max("value")) - val resultStage = resultDf2.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlan].finalPlan + val resultFragment = resultDf2.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan - // The result stage has 2 children - val level1Stages = resultStage.plan.collect { case q: QueryStage => q } - assert(level1Stages.length == 2) + // The result fragment has 2 children + val level1Fragments = resultFragment.plan.collect { case q: QueryFragmentExec => q } + assert(level1Fragments.length == 2) - val leafStages = level1Stages.flatMap { stage => - // All of the child stages of result stage have only one child stage. - val children = stage.plan.collect { case q: QueryStage => q } + val leafFragments = level1Fragments.flatMap { fragment => + // All of the child fragments of result fragment have only one child fragment. + val children = fragment.plan.collect { case q: QueryFragmentExec => q } assert(children.length == 1) children } - assert(leafStages.length == 2) + assert(leafFragments.length == 2) - val reusedStages = level1Stages.flatMap { stage => - stage.plan.collect { case r: ReusedQueryStage => r } + val reusedFragments = level1Fragments.flatMap { fragment => + fragment.plan.collect { case r: ReusedQueryFragmentExec => r } } - assert(reusedStages.length == 1) + assert(reusedFragments.length == 1) checkAnswer(resultDf2, Row(1, 0) :: Row(2, 0) :: Nil) } From e4bfc22b7ee15b6b737298dd3ed6d38d6e4e8996 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Wed, 27 Feb 2019 18:08:48 +0800 Subject: [PATCH 16/21] fix test --- .../spark/sql/execution/SparkPlanInfo.scala | 2 +- .../adaptive/QueryFragmentExec.scala | 2 +- .../AssertChildFragmentsMaterialized.scala | 4 ++-- .../rule/ReduceNumShufflePartitions.scala | 24 ++++++++++++++----- .../ReduceNumShufflePartitionsSuite.scala | 5 +++- 5 files changed, 26 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index fef74ea7cd3f4..0fc1358ec2525 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -54,7 +54,7 @@ private[execution] object SparkPlanInfo { val children = plan match { case ReusedExchangeExec(_, child) => child :: Nil case a: AdaptiveSparkPlanExec => a.finalPlan.plan :: Nil - case stage: QueryFragmentExec => stage.plan :: Nil + case fragment: QueryFragmentExec => fragment.plan :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala index bbc8722ea6590..cbad5710c4693 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala @@ -139,7 +139,7 @@ case class BroadcastQueryFragmentExec(id: Int, plan: BroadcastExchangeExec) * A wrapper of QueryFragment to indicate that it's reused. Note that this is not a query fragment. */ case class ReusedQueryFragmentExec(child: QueryFragmentExec, output: Seq[Attribute]) - extends UnaryExecNode { + extends LeafExecNode { // Ignore this wrapper for canonicalizing. override def doCanonicalize(): SparkPlan = child.canonicalized diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala index 97ba33e0e3a18..3d1d014cf4b88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala @@ -21,8 +21,8 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.QueryFragmentExec -// A sanity check rule to make sure we are running query stage optimizer rules on a sub-tree of -// query plan with all input stages materialized. +// A sanity check rule to make sure we are running query fragment optimizer rules on a sub-tree of +// query plan with all input fragments materialized. object AssertChildFragmentsMaterialized extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = plan.transform { case q: QueryFragmentExec if !q.materialize().isCompleted => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala index 8ad75a0c7adc3..51d67b8730950 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala @@ -17,17 +17,17 @@ package org.apache.spark.sql.execution.adaptive.rule -import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration.Duration import org.apache.spark.MapOutputStatistics +import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.adaptive.{QueryFragmentExec, ShuffleQueryFragmentExec} +import org.apache.spark.sql.execution.adaptive.{QueryFragmentExec, ReusedQueryFragmentExec, ShuffleQueryFragmentExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ThreadUtils @@ -61,7 +61,9 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { ThreadUtils.awaitResult(metricsFuture, Duration.Zero) } - val allFragmentLeaves = plan.collectLeaves().forall(_.isInstanceOf[QueryFragmentExec]) + val allFragmentLeaves = plan.collectLeaves().forall { node => + node.isInstanceOf[QueryFragmentExec] || node.isInstanceOf[ReusedQueryFragmentExec] + } if (allFragmentLeaves) { // ShuffleQueryFragment gives null mapOutputStatistics when the input RDD has 0 partitions, @@ -76,6 +78,8 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // number of output partitions. case fragment: ShuffleQueryFragmentExec => CoalescedShuffleReaderExec(fragment, partitionStartIndices) + case r@ReusedQueryFragmentExec(fragment: ShuffleQueryFragmentExec, output) => + CoalescedShuffleReaderExec(r, partitionStartIndices) } } else { plan @@ -152,7 +156,9 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { partitionStartIndices += i // reset postShuffleInputSize. postShuffleInputSize = nextShuffleInputSize - } else postShuffleInputSize += nextShuffleInputSize + } else { + postShuffleInputSize += nextShuffleInputSize + } i += 1 } @@ -162,7 +168,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { } case class CoalescedShuffleReaderExec( - child: ShuffleQueryFragmentExec, + child: SparkPlan, partitionStartIndices: Array[Int]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output @@ -175,7 +181,13 @@ case class CoalescedShuffleReaderExec( override protected def doExecute(): RDD[InternalRow] = { if (cachedShuffleRDD == null) { - cachedShuffleRDD = child.plan.createShuffledRDD(Some(partitionStartIndices)) + cachedShuffleRDD = child match { + case fragment: ShuffleQueryFragmentExec => + fragment.plan.createShuffledRDD(Some(partitionStartIndices)) + case ReusedQueryFragmentExec(fragment: ShuffleQueryFragmentExec, _) => + fragment.plan.createShuffledRDD(Some(partitionStartIndices)) + case _ => throw new SparkException("Invalid child for CoalescedShuffleReaderExec") + } } 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 4e268663569cd..688e0d06cdb89 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 @@ -558,7 +558,10 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val leafFragments = level1Fragments.flatMap { fragment => // All of the child fragments of result fragment have only one child fragment. - val children = fragment.plan.collect { case q: QueryFragmentExec => q } + val children = fragment.plan.collect { + case q: QueryFragmentExec => q + case r: ReusedQueryFragmentExec => r.child + } assert(children.length == 1) children } From bef8ab87961424717249f2a251c9164b364c8e87 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 28 Feb 2019 11:04:45 +0800 Subject: [PATCH 17/21] add import --- .../sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala index 51d67b8730950..31edd21ff3f79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.adaptive.rule +import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration.Duration import org.apache.spark.MapOutputStatistics From 2d6f110ea34da4f3cc551cb49d2bd23cc5d9f028 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 28 Feb 2019 16:42:51 +0800 Subject: [PATCH 18/21] enable AE for testing --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c5a728cd3f319..34ddb34237250 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 @@ -278,7 +278,7 @@ object SQLConf { val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled") .doc("When true, enable adaptive query execution.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS = buildConf("spark.sql.adaptive.minNumPostShufflePartitions") From fd413d4f7532d95b39b8b601e7b7c890c85645d9 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 15 Mar 2019 15:21:40 +0800 Subject: [PATCH 19/21] improve (#9) --- .../spark/sql/execution/SparkPlanInfo.scala | 6 +- .../adaptive/AdaptiveSparkPlanExec.scala | 60 ++-- .../adaptive/QueryFragmentCreator.scala | 260 ------------------ ...ragmentExec.scala => QueryStageExec.scala} | 63 ++--- .../adaptive/QueryStageManager.scala | 220 +++++++++++++++ ...la => AssertChildStagesMaterialized.scala} | 12 +- .../rule/ReduceNumShufflePartitions.scala | 41 +-- .../rule/RemoveRedundantShuffles.scala | 39 +++ .../ReduceNumShufflePartitionsSuite.scala | 25 +- 9 files changed, 351 insertions(+), 375 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentCreator.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/{QueryFragmentExec.scala => QueryStageExec.scala} (67%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/{AssertChildFragmentsMaterialized.scala => AssertChildStagesMaterialized.scala} (70%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/RemoveRedundantShuffles.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 0fc1358ec2525..21ef289fa18f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryFragmentExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo import org.apache.spark.sql.internal.SQLConf @@ -53,8 +53,8 @@ private[execution] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val children = plan match { case ReusedExchangeExec(_, child) => child :: Nil - case a: AdaptiveSparkPlanExec => a.finalPlan.plan :: Nil - case fragment: QueryFragmentExec => fragment.plan :: Nil + case a: AdaptiveSparkPlanExec => a.finalPlan :: Nil + case fragment: QueryStageExec => fragment.plan :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => 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 7da258842286b..ae6290782fe31 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 @@ -28,9 +28,11 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, SparkPlanInfo, S import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate /** - * A root node to execute the query plan adaptively. It creates query fragments, and incrementally - * updates the query plan when a query fragment is materialized and provides accurate runtime - * data statistics. + * A root node to execute the query plan adaptively. It splits the query plan into independent + * stages and executes them in order according to their dependencies. The query stage + * materializes its output at the end. When one stage completes, the data statistics of its + * materialized output will be used to optimize the subsequent stages. + * This is called mid-query re-optimization in database literature. */ case class AdaptiveSparkPlanExec(initialPlan: SparkPlan, session: SparkSession) extends LeafExecNode{ @@ -40,34 +42,34 @@ case class AdaptiveSparkPlanExec(initialPlan: SparkPlan, session: SparkSession) @volatile private var currentPlan: SparkPlan = initialPlan @volatile private var error: Throwable = null - // We will release the lock when we finish planning query fragments, or we fail to do the - // planning. Getting `finalPlan` will be blocked until the lock is release. + // We will release the lock when all the query stages are completed, or we fail to + // optimize/execute query stages. Getting `finalPlan` will be blocked until the lock is release. // This is better than wait()/notify(), as we can easily check if the computation has completed, // by calling `readyLock.getCount()`. private val readyLock = new CountDownLatch(1) - private def createCallback(executionId: Option[Long]): QueryFragmentCreatorCallback = { - new QueryFragmentCreatorCallback { - override def onPlanUpdate(updatedPlan: SparkPlan): Unit = { - updateCurrentPlan(updatedPlan, executionId) - if (updatedPlan.isInstanceOf[ResultQueryFragmentExec]) readyLock.countDown() - } + private def createCallback(executionId: Option[Long]) = new QueryStageManagerCallback { + override def onPlanUpdate(updatedPlan: SparkPlan): Unit = { + updateCurrentPlan(updatedPlan, executionId) + } + + override def onFinalPlan(finalPlan: SparkPlan): Unit = { + updateCurrentPlan(finalPlan, executionId) + readyLock.countDown() + } - override def onFragmentMaterializingFailed( - fragment: QueryFragmentExec, - e: Throwable): Unit = { - error = new SparkException( - s""" - |Fail to materialize fragment ${fragment.id}: - |${fragment.plan.treeString} + override def onStageMaterializationFailed(stage: QueryStageExec, e: Throwable): Unit = { + error = new SparkException( + s""" + |Fail to materialize query stage ${stage.id}: + |${stage.plan.treeString} """.stripMargin, e) - readyLock.countDown() - } + readyLock.countDown() + } - override def onError(e: Throwable): Unit = { - error = e - readyLock.countDown() - } + override def onError(e: Throwable): Unit = { + error = e + readyLock.countDown() } } @@ -81,18 +83,18 @@ case class AdaptiveSparkPlanExec(initialPlan: SparkPlan, session: SparkSession) } } - def finalPlan: ResultQueryFragmentExec = { + def finalPlan: SparkPlan = { if (readyLock.getCount > 0) { val sc = session.sparkContext val executionId = Option(sc.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).map(_.toLong) - val creator = new QueryFragmentCreator(initialPlan, session, createCallback(executionId)) - creator.start() + val stageManager = new QueryStageManager(initialPlan, session, createCallback(executionId)) + stageManager.start() readyLock.await() - creator.stop() + stageManager.stop() } if (error != null) throw error - currentPlan.asInstanceOf[ResultQueryFragmentExec] + currentPlan } override def executeCollect(): Array[InternalRow] = finalPlan.executeCollect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentCreator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentCreator.scala deleted file mode 100644 index 49f7a271872b3..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentCreator.scala +++ /dev/null @@ -1,260 +0,0 @@ -/* - * 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.{ExecutionContext, ExecutionContextExecutorService} - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{CollapseCodegenStages, SparkPlan} -import org.apache.spark.sql.execution.adaptive.rule.{AssertChildFragmentsMaterialized, ReduceNumShufflePartitions} -import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ShuffleExchangeExec} -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.{EventLoop, ThreadUtils} - -/** - * This class dynamically creates [[QueryFragmentExec]] bottom-up, optimize the query plan of query - * fragments and materialize them. It creates as many query fragments as possible at the same time, - * and materialize a query fragment when all its child fragments are materialized. - * - * To create query fragments, we traverse the query tree bottom up. When we hit an exchange node, - * and all the child query fragments of this exchange node are materialized, we try to create a new - * query fragment for this exchange node. - * - * To create a new query fragment, we first optimize the sub-tree of the exchange. After - * optimization, we check the output partitioning of the optimized sub-tree, and see if the - * exchange node is still necessary. - * - * If the exchange node becomes unnecessary, remove it and give up this query fragment creation, - * and continue to traverse the query plan tree until we hit the next exchange node. - * - * If the exchange node is still needed, create the query fragment and optimize its sub-tree again. - * It's necessary to have both the pre-creation optimization and post-creation optimization, because - * these 2 optimization have different assumptions. For pre-creation optimization, the shuffle node - * may be removed later on and the current sub-tree may be only a part of a query fragment, so we - * don't have the big picture of the query fragment yet. For post-creation optimization, the query - * fragment is created and we have the big picture of the query fragment. - * - * After the query fragment is optimized, we materialize it asynchronously, and continue to traverse - * the query plan tree to create more query fragments. - * - * When a query fragment completes materialization, we trigger the process of query fragments - * creation and traverse the query plan tree again. - */ -class QueryFragmentCreator( - initialPlan: SparkPlan, - session: SparkSession, - callback: QueryFragmentCreatorCallback) - extends EventLoop[QueryFragmentCreatorEvent]("QueryFragmentCreator") { - - private def conf = session.sessionState.conf - - private val readyFragments = mutable.HashSet.empty[Int] - - private var currentFragmentId = 0 - - private val fragmentCache = - mutable.HashMap.empty[StructType, mutable.Buffer[(Exchange, QueryFragmentExec)]] - - // The optimizer rules that will be applied to a sub-tree of the query plan before the fragment is - // created. Note that we may end up not creating the query fragment, so the rules here should not - // assume the given sub-plan-tree is the entire query plan of the query fragment. For example, if - // a rule want to collect all the child query fragments, it should not be put here. - private val preFragmentCreationOptimizerRules: Seq[Rule[SparkPlan]] = Seq( - AssertChildFragmentsMaterialized - ) - - // The optimizer rules that will be applied to a sub-tree of the query plan after the fragment is - // created. Note that once the fragment is created, we will not remove it anymore. If a rule - // changes the output partitioning of the sub-plan-tree, which may help to remove the exchange - // node, it's better to put it in `preFragmentCreationOptimizerRules`, so that we may create less - // query fragments. - private val postFragmentCreationOptimizerRules: Seq[Rule[SparkPlan]] = Seq( - ReduceNumShufflePartitions(conf), - CollapseCodegenStages(conf)) - - private var currentPlan = initialPlan - - private val localProperties = session.sparkContext.getLocalProperties - - private implicit def executionContext: ExecutionContextExecutorService = { - QueryFragmentCreator.executionContext - } - - override protected def onReceive(event: QueryFragmentCreatorEvent): Unit = event match { - case StartCreation => - // set active session and local properties for the event loop thread. - SparkSession.setActiveSession(session) - session.sparkContext.setLocalProperties(localProperties) - currentPlan = createQueryFragments(initialPlan) - - case MaterializeFragment(fragment) => - fragment.materialize().onComplete { res => - if (res.isSuccess) { - post(FragmentReady(fragment)) - } else { - callback.onFragmentMaterializingFailed(fragment, res.failed.get) - stop() - } - } - - case FragmentReady(fragment) => - if (fragment.isInstanceOf[ResultQueryFragmentExec]) { - callback.onPlanUpdate(fragment) - stop() - } else { - readyFragments += fragment.id - currentPlan = createQueryFragments(currentPlan) - } - } - - override protected def onStart(): Unit = { - post(StartCreation) - } - - private def preFragmentCreationOptimize(plan: SparkPlan): SparkPlan = { - preFragmentCreationOptimizerRules.foldLeft(plan) { - case (current, rule) => rule(current) - } - } - - private def postFragmentCreationOptimize(plan: SparkPlan): SparkPlan = { - postFragmentCreationOptimizerRules.foldLeft(plan) { - case (current, rule) => rule(current) - } - } - - /** - * Traverse the query plan bottom-up, and creates query fragments as many as possible. - */ - private def createQueryFragments(plan: SparkPlan): SparkPlan = { - val result = createQueryFragments0(plan) - if (result.allChildFragmentsReady) { - val finalPlan = postFragmentCreationOptimize(preFragmentCreationOptimize(result.newPlan)) - post(FragmentReady(ResultQueryFragmentExec(currentFragmentId, finalPlan))) - finalPlan - } else { - callback.onPlanUpdate(result.newPlan) - result.newPlan - } - } - - /** - * This method is called recursively to traverse the plan tree bottom-up. This method returns two - * information: 1) the new plan after we insert query fragments. 2) whether or not the child query - * fragments of the new plan are all ready. - * - * if the current plan is an exchange node, and all its child query fragments are ready, we try to - * create a new query fragment. - */ - private def createQueryFragments0(plan: SparkPlan): CreateFragmentResult = plan match { - case e: Exchange => - val similarFragments = fragmentCache.getOrElseUpdate(e.schema, mutable.Buffer.empty) - similarFragments.find(_._1.sameResult(e)) match { - case Some((_, existingFragment)) if conf.exchangeReuseEnabled => - CreateFragmentResult( - newPlan = ReusedQueryFragmentExec(existingFragment, e.output), - allChildFragmentsReady = readyFragments.contains(existingFragment.id)) - - case _ => - val result = createQueryFragments0(e.child) - // Try to create a query fragment only when all the child query fragments are ready. - if (result.allChildFragmentsReady) { - val optimizedPlan = preFragmentCreationOptimize(result.newPlan) - e match { - case s: ShuffleExchangeExec => - (s.desiredPartitioning, optimizedPlan.outputPartitioning) match { - case (desired: HashPartitioning, actual: HashPartitioning) - if desired.semanticEquals(actual) => - // This shuffle exchange is unnecessary now, remove it. The reason maybe: - // 1. the child plan has changed its output partitioning after optimization, - // and makes this exchange node unnecessary. - // 2. this exchange node is user specified, which turns out to be unnecessary. - CreateFragmentResult(newPlan = optimizedPlan, allChildFragmentsReady = true) - case _ => - val queryFragment = createQueryFragment(s.copy(child = optimizedPlan)) - similarFragments.append(e -> queryFragment) - // We've created a new fragment, which is obviously not ready yet. - CreateFragmentResult(newPlan = queryFragment, allChildFragmentsReady = false) - } - - case b: BroadcastExchangeExec => - val queryFragment = createQueryFragment(b.copy(child = optimizedPlan)) - similarFragments.append(e -> queryFragment) - // We've created a new fragment, which is obviously not ready yet. - CreateFragmentResult(newPlan = queryFragment, allChildFragmentsReady = false) - } - } else { - CreateFragmentResult( - newPlan = e.withNewChildren(Seq(result.newPlan)), - allChildFragmentsReady = false) - } - } - - case q: QueryFragmentExec => - CreateFragmentResult(newPlan = q, allChildFragmentsReady = readyFragments.contains(q.id)) - - case _ => - if (plan.children.isEmpty) { - CreateFragmentResult(newPlan = plan, allChildFragmentsReady = true) - } else { - val results = plan.children.map(createQueryFragments0) - CreateFragmentResult( - newPlan = plan.withNewChildren(results.map(_.newPlan)), - allChildFragmentsReady = results.forall(_.allChildFragmentsReady)) - } - } - - private def createQueryFragment(e: Exchange): QueryFragmentExec = { - val optimizedPlan = postFragmentCreationOptimize(e.child) - val queryFragment = e match { - case s: ShuffleExchangeExec => - ShuffleQueryFragmentExec(currentFragmentId, s.copy(child = optimizedPlan)) - case b: BroadcastExchangeExec => - BroadcastQueryFragmentExec(currentFragmentId, b.copy(child = optimizedPlan)) - } - currentFragmentId += 1 - post(MaterializeFragment(queryFragment)) - queryFragment - } - - override protected def onError(e: Throwable): Unit = callback.onError(e) -} - -case class CreateFragmentResult(newPlan: SparkPlan, allChildFragmentsReady: Boolean) - -object QueryFragmentCreator { - private val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("QueryFragmentCreator", 16)) -} - -trait QueryFragmentCreatorCallback { - def onPlanUpdate(updatedPlan: SparkPlan): Unit - def onFragmentMaterializingFailed(fragment: QueryFragmentExec, e: Throwable): Unit - def onError(e: Throwable): Unit -} - -sealed trait QueryFragmentCreatorEvent - -object StartCreation extends QueryFragmentCreatorEvent - -case class MaterializeFragment(fragment: QueryFragmentExec) extends QueryFragmentCreatorEvent - -case class FragmentReady(fragment: QueryFragmentExec) extends QueryFragmentCreatorEvent diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala similarity index 67% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index cbad5710c4693..80c623ea94400 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryFragmentExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -29,32 +29,38 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange._ /** - * A query fragment is an individual sub-tree of a query plan, which can be executed ahead and - * provide accurate data statistics. For example, a sub-tree under shuffle/broadcast node is a - * query fragment. Each query fragment runs in a single Spark job/stage. + * A query stage is an independent subgraph of the query plan. Query stage materializes its output + * before proceeding with further operators of the query plan. The data statistics of the + * materialized output can be used to optimize subsequent query stages. + * + * There are 2 kinds of query stages: + * 1. Shuffle query stage. This stage materializes its output to shuffle files, and Spark launches + * another job to execute the further operators. + * 2. Broadcast stage. This stage materializes its output to an array in driver JVM. Spark + * broadcasts the array before executing the further operators. */ -abstract class QueryFragmentExec extends LeafExecNode { +abstract class QueryStageExec extends LeafExecNode { /** - * An id of this query fragment which is unique in the entire query plan. + * An id of this query stage which is unique in the entire query plan. */ def id: Int /** - * The sub-tree of the query plan that belongs to this query fragment. + * The sub-tree of the query plan that belongs to this query stage. */ def plan: SparkPlan /** - * Returns a new query fragment with a new plan, which is optimized based on accurate runtime + * Returns a new query stage with a new plan, which is optimized based on accurate runtime data * statistics. */ - def withNewPlan(newPlan: SparkPlan): QueryFragmentExec + def withNewPlan(newPlan: SparkPlan): QueryStageExec /** - * Materialize this QueryFragment, to prepare for the execution, like submitting map stages, + * 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 - * fragment is ready. + * stage is ready. */ def materialize(): Future[Any] @@ -68,7 +74,7 @@ abstract class QueryFragmentExec extends LeafExecNode { override def doExecuteBroadcast[T](): Broadcast[T] = plan.executeBroadcast() override def doCanonicalize(): SparkPlan = plan.canonicalized - // TODO: maybe we should not hide QueryFragment entirely from explain result. + // TODO: maybe we should not hide query stage entirely from explain result. override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], @@ -83,25 +89,11 @@ abstract class QueryFragmentExec extends LeafExecNode { } /** - * The last QueryFragment of an execution plan. - */ -case class ResultQueryFragmentExec(id: Int, plan: SparkPlan) extends QueryFragmentExec { - - override def materialize(): Future[Any] = { - throw new IllegalStateException("Cannot materialize ResultQueryFragment.") - } - - override def withNewPlan(newPlan: SparkPlan): QueryFragmentExec = { - copy(plan = newPlan) - } -} - -/** - * A shuffle QueryFragment whose child is a [[ShuffleExchangeExec]]. + * A shuffle query stage whose child is a [[ShuffleExchangeExec]]. */ -case class ShuffleQueryFragmentExec(id: Int, plan: ShuffleExchangeExec) extends QueryFragmentExec { +case class ShuffleQueryStageExec(id: Int, plan: ShuffleExchangeExec) extends QueryStageExec { - override def withNewPlan(newPlan: SparkPlan): QueryFragmentExec = { + override def withNewPlan(newPlan: SparkPlan): QueryStageExec = { copy(plan = newPlan.asInstanceOf[ShuffleExchangeExec]) } @@ -121,12 +113,11 @@ case class ShuffleQueryFragmentExec(id: Int, plan: ShuffleExchangeExec) extends } /** - * A broadcast QueryFragment whose child is a [[BroadcastExchangeExec]]. + * A broadcast query stage whose child is a [[BroadcastExchangeExec]]. */ -case class BroadcastQueryFragmentExec(id: Int, plan: BroadcastExchangeExec) - extends QueryFragmentExec { +case class BroadcastQueryStageExec(id: Int, plan: BroadcastExchangeExec) extends QueryStageExec { - override def withNewPlan(newPlan: SparkPlan): QueryFragmentExec = { + override def withNewPlan(newPlan: SparkPlan): QueryStageExec = { copy(plan = newPlan.asInstanceOf[BroadcastExchangeExec]) } @@ -136,10 +127,10 @@ case class BroadcastQueryFragmentExec(id: Int, plan: BroadcastExchangeExec) } /** - * A wrapper of QueryFragment to indicate that it's reused. Note that this is not a query fragment. + * A wrapper of query stage to indicate that it's reused. Note that itself is not a query stage. */ -case class ReusedQueryFragmentExec(child: QueryFragmentExec, output: Seq[Attribute]) - extends LeafExecNode { +case class ReusedQueryStageExec(child: SparkPlan, output: Seq[Attribute]) + extends UnaryExecNode { // Ignore this wrapper for canonicalizing. override def doCanonicalize(): SparkPlan = child.canonicalized @@ -152,7 +143,7 @@ case class ReusedQueryFragmentExec(child: QueryFragmentExec, output: Seq[Attribu child.executeBroadcast() } - // `ReusedQueryFragment` can have distinct set of output attribute ids from its child, we need + // `ReusedQueryStageExec` can have distinct set of output attribute ids from its child, we need // to update the attribute ids in `outputPartitioning` and `outputOrdering`. private lazy val updateAttr: Expression => Expression = { val originalAttrToNewAttr = AttributeMap(child.output.zip(output)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala new file mode 100644 index 0000000000000..49a602af0a71a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala @@ -0,0 +1,220 @@ +/* + * 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.{ExecutionContext, ExecutionContextExecutorService} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{CollapseCodegenStages, SparkPlan} +import org.apache.spark.sql.execution.adaptive.rule._ +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ShuffleExchangeExec} +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.{EventLoop, ThreadUtils} + +/** + * This class inserts [[QueryStageExec]] into the query plan in a bottom-up fashion, and + * materializes the query stages asynchronously as soon as they are created. + * + * When one query stage finishes materialization, a list of adaptive optimizer rules will be + * executed, trying to optimize the query plan with the data statistics collected from the the + * materialized data. Then we travers the query plan again and try to insert more query stages. + * + * To create query stages, we traverse the query tree bottom up. When we hit an exchange node, + * and all the child query stages of this exchange node are materialized, we create a new + * query stage for this exchange node. + * + * Right before the stage creation, a list of query stage optimizer rules will be executed. These + * optimizer rules are different from the adaptive optimizer rules. Query stage optimizer rules only + * focus on a plan sub-tree of a specific query stage, and they will be executed only after all the + * child stages are materialized. + */ +class QueryStageManager( + initialPlan: SparkPlan, + session: SparkSession, + callback: QueryStageManagerCallback) + extends EventLoop[QueryStageManagerEvent]("QueryFragmentCreator") { + + private def conf = session.sessionState.conf + + private val readyStages = mutable.HashSet.empty[Int] + + private var currentStageId = 0 + + private val stageCache = + mutable.HashMap.empty[StructType, mutable.Buffer[(Exchange, QueryStageExec)]] + + private var currentPlan = initialPlan + + private val localProperties = session.sparkContext.getLocalProperties + + private implicit def executionContext: ExecutionContextExecutorService = { + QueryStageManager.executionContext + } + + // A list of optimizer rules that will be applied when a query stage finishes materialization. + // These rules need to travers the entire query plan, and find chances to optimize the query plan + // with the data statistics collected from materialized query stage's output. + private val adaptiveOptimizerRules: Seq[Rule[SparkPlan]] = Seq( + RemoveRedundantShuffles) + + // A list of optimizer rules that will be applied right before a query stage is created. + // These rules need to traverse the plan sub-tree of the query stage to be created, and find + // chances to optimize this query stage given the all its child query stages. + private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( + AssertChildStagesMaterialized, + ReduceNumShufflePartitions(conf), + CollapseCodegenStages(conf)) + + private def optimizeEntirePlan(plan: SparkPlan): SparkPlan = { + adaptiveOptimizerRules.foldLeft(plan) { + case (current, rule) => rule(current) + } + } + + private def optimizeQueryStage(plan: SparkPlan): SparkPlan = { + queryStageOptimizerRules.foldLeft(plan) { + case (current, rule) => rule(current) + } + } + + override protected def onReceive(event: QueryStageManagerEvent): Unit = event match { + case Start => + // set active session and local properties for the event loop thread. + SparkSession.setActiveSession(session) + session.sparkContext.setLocalProperties(localProperties) + currentPlan = createQueryStages(initialPlan) + + case MaterializeStage(stage) => + stage.materialize().onComplete { res => + if (res.isSuccess) { + post(StageReady(stage)) + } else { + callback.onStageMaterializationFailed(stage, res.failed.get) + stop() + } + } + + case StageReady(stage) => + readyStages += stage.id + currentPlan = optimizeEntirePlan(currentPlan) + currentPlan = createQueryStages(currentPlan) + } + + override protected def onStart(): Unit = { + post(Start) + } + + /** + * Traverse the query plan bottom-up, and creates query stages as many as possible. + */ + private def createQueryStages(plan: SparkPlan): SparkPlan = { + val result = createQueryStages0(plan) + if (result.allChildStagesReady) { + val finalPlan = optimizeQueryStage(result.newPlan) + callback.onFinalPlan(finalPlan) + finalPlan + } else { + callback.onPlanUpdate(result.newPlan) + result.newPlan + } + } + + /** + * This method is called recursively to traverse the plan tree bottom-up. This method returns two + * information: 1) the new plan after we insert query stages. 2) whether or not the child query + * stages of the new plan are all ready. + * + * if the current plan is an exchange node, and all its child query stages are ready, we create + * a new query stage. + */ + private def createQueryStages0(plan: SparkPlan): CreateStageResult = plan match { + case e: Exchange => + val similarStages = stageCache.getOrElseUpdate(e.schema, mutable.Buffer.empty) + similarStages.find(_._1.sameResult(e)) match { + case Some((_, existingStage)) if conf.exchangeReuseEnabled => + CreateStageResult( + newPlan = ReusedQueryStageExec(existingStage, e.output), + allChildStagesReady = readyStages.contains(existingStage.id)) + + case _ => + val result = createQueryStages0(e.child) + val newPlan = e.withNewChildren(Seq(result.newPlan)).asInstanceOf[Exchange] + // Create a query stage only when all the child query stages are ready. + if (result.allChildStagesReady) { + val queryStage = createQueryStage(newPlan) + similarStages.append(e -> queryStage) + // We've created a new stage, which is obviously not ready yet. + CreateStageResult(newPlan = queryStage, allChildStagesReady = false) + } else { + CreateStageResult(newPlan = newPlan, allChildStagesReady = false) + } + } + + case q: QueryStageExec => + CreateStageResult(newPlan = q, allChildStagesReady = readyStages.contains(q.id)) + + case _ => + if (plan.children.isEmpty) { + CreateStageResult(newPlan = plan, allChildStagesReady = true) + } else { + val results = plan.children.map(createQueryStages0) + CreateStageResult( + newPlan = plan.withNewChildren(results.map(_.newPlan)), + allChildStagesReady = results.forall(_.allChildStagesReady)) + } + } + + private def createQueryStage(e: Exchange): QueryStageExec = { + val optimizedPlan = optimizeQueryStage(e.child) + val queryStage = e match { + case s: ShuffleExchangeExec => + ShuffleQueryStageExec(currentStageId, s.copy(child = optimizedPlan)) + case b: BroadcastExchangeExec => + BroadcastQueryStageExec(currentStageId, b.copy(child = optimizedPlan)) + } + currentStageId += 1 + post(MaterializeStage(queryStage)) + queryStage + } + + override protected def onError(e: Throwable): Unit = callback.onError(e) +} + +case class CreateStageResult(newPlan: SparkPlan, allChildStagesReady: Boolean) + +object QueryStageManager { + private val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("QueryFragmentCreator", 16)) +} + +trait QueryStageManagerCallback { + def onPlanUpdate(updatedPlan: SparkPlan): Unit + def onFinalPlan(finalPlan: SparkPlan): Unit + def onStageMaterializationFailed(stage: QueryStageExec, e: Throwable): Unit + def onError(e: Throwable): Unit +} + +sealed trait QueryStageManagerEvent + +object Start extends QueryStageManagerEvent + +case class MaterializeStage(stage: QueryStageExec) extends QueryStageManagerEvent + +case class StageReady(stage: QueryStageExec) extends QueryStageManagerEvent diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala similarity index 70% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala index 3d1d014cf4b88..f522c12dd663c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildFragmentsMaterialized.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/AssertChildStagesMaterialized.scala @@ -19,14 +19,14 @@ package org.apache.spark.sql.execution.adaptive.rule import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.adaptive.QueryFragmentExec +import org.apache.spark.sql.execution.adaptive.QueryStageExec -// A sanity check rule to make sure we are running query fragment optimizer rules on a sub-tree of -// query plan with all input fragments materialized. -object AssertChildFragmentsMaterialized extends Rule[SparkPlan] { +// A sanity check rule to make sure we are running query stage optimizer rules on a sub-tree of +// query plan with all child query stages materialized. +object AssertChildStagesMaterialized extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = plan.transform { - case q: QueryFragmentExec if !q.materialize().isCompleted => + case q: QueryStageExec if !q.materialize().isCompleted => throw new IllegalArgumentException( - s"The input fragments should all be materialized, but the below one is not.\n ${q.plan}") + s"The input query stages should all be materialized, but the below one is not.\n ${q.plan}") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala index 31edd21ff3f79..9a372bbb6a8be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala @@ -21,14 +21,13 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration.Duration import org.apache.spark.MapOutputStatistics -import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.adaptive.{QueryFragmentExec, ReusedQueryFragmentExec, ShuffleQueryFragmentExec} +import org.apache.spark.sql.execution.adaptive.{QueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ThreadUtils @@ -56,18 +55,19 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { val shuffleMetrics: Seq[MapOutputStatistics] = plan.collect { - case fragment: ShuffleQueryFragmentExec => + case fragment: ShuffleQueryStageExec => val metricsFuture = fragment.mapOutputStatisticsFuture assert(metricsFuture.isCompleted, "ShuffleQueryFragment should already be ready") ThreadUtils.awaitResult(metricsFuture, Duration.Zero) } - val allFragmentLeaves = plan.collectLeaves().forall { node => - node.isInstanceOf[QueryFragmentExec] || node.isInstanceOf[ReusedQueryFragmentExec] - } - - if (allFragmentLeaves) { - // ShuffleQueryFragment gives null mapOutputStatistics when the input RDD has 0 partitions, + if (!plan.collectLeaves().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. + plan + } else { + // `ShuffleQueryStageExec` gives null mapOutputStatistics when the input RDD has 0 partitions, // we should skip it when calculating the `partitionStartIndices`. val validMetrics = shuffleMetrics.filter(_ != null) if (validMetrics.nonEmpty) { @@ -75,21 +75,14 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // 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 fragment have the same + // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same // number of output partitions. - case fragment: ShuffleQueryFragmentExec => - CoalescedShuffleReaderExec(fragment, partitionStartIndices) - case r@ReusedQueryFragmentExec(fragment: ShuffleQueryFragmentExec, output) => - CoalescedShuffleReaderExec(r, partitionStartIndices) + case stage: ShuffleQueryStageExec => + CoalescedShuffleReaderExec(stage, partitionStartIndices) } } else { plan } - } else { - // If not all leaf nodes are query fragments, 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. - plan } } @@ -169,7 +162,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { } case class CoalescedShuffleReaderExec( - child: SparkPlan, + child: ShuffleQueryStageExec, partitionStartIndices: Array[Int]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output @@ -182,13 +175,7 @@ case class CoalescedShuffleReaderExec( override protected def doExecute(): RDD[InternalRow] = { if (cachedShuffleRDD == null) { - cachedShuffleRDD = child match { - case fragment: ShuffleQueryFragmentExec => - fragment.plan.createShuffledRDD(Some(partitionStartIndices)) - case ReusedQueryFragmentExec(fragment: ShuffleQueryFragmentExec, _) => - fragment.plan.createShuffledRDD(Some(partitionStartIndices)) - case _ => throw new SparkException("Invalid child for CoalescedShuffleReaderExec") - } + cachedShuffleRDD = child.plan.createShuffledRDD(Some(partitionStartIndices)) } cachedShuffleRDD } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/RemoveRedundantShuffles.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/RemoveRedundantShuffles.scala new file mode 100644 index 0000000000000..1112869399d5b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/RemoveRedundantShuffles.scala @@ -0,0 +1,39 @@ +/* + * 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.rule + +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec + +/** + * Remove shuffle nodes if the child's output partitions is already the desired partitioning. + * + * This should be the last rule of adaptive optimizer rules, as other rules may change plan + * node's output partitioning and make some shuffle nodes become unnecessary. + */ +object RemoveRedundantShuffles extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transformUp { + case shuffle @ ShuffleExchangeExec(upper: HashPartitioning, child) => + child.outputPartitioning match { + case lower: HashPartitioning if upper.semanticEquals(lower) => child + case _ => shuffle + } + } +} 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 688e0d06cdb89..83deae2342a37 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 @@ -315,7 +315,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = agg.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -362,7 +362,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -414,7 +414,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -466,7 +466,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -509,7 +509,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Then, let's make sure we do not reduce number of ppst shuffle partitions. val finalPlan = join.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan val shuffleReaders = finalPlan.collect { case reader: CoalescedShuffleReaderExec => reader } @@ -534,8 +534,8 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // ReusedQueryFragment 0 val resultDf = df.join(df, "key").join(df, "key") val finalPlan = resultDf.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec].finalPlan.plan - assert(finalPlan.collect { case p: ReusedQueryFragmentExec => p }.length == 2) + .asInstanceOf[AdaptiveSparkPlanExec].finalPlan + assert(finalPlan.collect { case p: ReusedQueryStageExec => p }.length == 2) assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 3) checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) @@ -549,26 +549,23 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val resultDf2 = grouped.groupBy(col("key") + 1).max("value") .union(grouped.groupBy(col("key") + 2).max("value")) - val resultFragment = resultDf2.queryExecution.executedPlan + val finalPlan2 = resultDf2.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].finalPlan // The result fragment has 2 children - val level1Fragments = resultFragment.plan.collect { case q: QueryFragmentExec => q } + val level1Fragments = finalPlan2.collect { case q: QueryStageExec => q } assert(level1Fragments.length == 2) val leafFragments = level1Fragments.flatMap { fragment => // All of the child fragments of result fragment have only one child fragment. - val children = fragment.plan.collect { - case q: QueryFragmentExec => q - case r: ReusedQueryFragmentExec => r.child - } + val children = fragment.plan.collect { case q: QueryStageExec => q } assert(children.length == 1) children } assert(leafFragments.length == 2) val reusedFragments = level1Fragments.flatMap { fragment => - fragment.plan.collect { case r: ReusedQueryFragmentExec => r } + fragment.plan.collect { case r: ReusedQueryStageExec => r } } assert(reusedFragments.length == 1) From 028b0ac8fb31b0f12f2b65a548b4fd623b4088c9 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Fri, 15 Mar 2019 15:49:43 +0800 Subject: [PATCH 20/21] rename --- .../apache/spark/sql/execution/SparkPlanInfo.scala | 2 +- .../sql/execution/adaptive/QueryStageManager.scala | 4 ++-- .../adaptive/rule/ReduceNumShufflePartitions.scala | 12 ++++++------ .../spark/sql/execution/ui/SparkPlanGraph.scala | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 21ef289fa18f1..f59eba6a51fc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -54,7 +54,7 @@ private[execution] object SparkPlanInfo { val children = plan match { case ReusedExchangeExec(_, child) => child :: Nil case a: AdaptiveSparkPlanExec => a.finalPlan :: Nil - case fragment: QueryStageExec => fragment.plan :: Nil + case stage: QueryStageExec => stage.plan :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala index 49a602af0a71a..a98403b6727f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala @@ -49,7 +49,7 @@ class QueryStageManager( initialPlan: SparkPlan, session: SparkSession, callback: QueryStageManagerCallback) - extends EventLoop[QueryStageManagerEvent]("QueryFragmentCreator") { + extends EventLoop[QueryStageManagerEvent]("QueryStageCreator") { private def conf = session.sessionState.conf @@ -201,7 +201,7 @@ case class CreateStageResult(newPlan: SparkPlan, allChildStagesReady: Boolean) object QueryStageManager { private val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("QueryFragmentCreator", 16)) + ThreadUtils.newDaemonCachedThreadPool("QueryStageCreator", 16)) } trait QueryStageManagerCallback { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala index 9a372bbb6a8be..c61380bd10d6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/rule/ReduceNumShufflePartitions.scala @@ -55,9 +55,9 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { val shuffleMetrics: Seq[MapOutputStatistics] = plan.collect { - case fragment: ShuffleQueryStageExec => - val metricsFuture = fragment.mapOutputStatisticsFuture - assert(metricsFuture.isCompleted, "ShuffleQueryFragment should already be ready") + case stage: ShuffleQueryStageExec => + val metricsFuture = stage.mapOutputStatisticsFuture + assert(metricsFuture.isCompleted, "ShuffleQueryStageExec should already be ready") ThreadUtils.awaitResult(metricsFuture, Duration.Zero) } @@ -88,7 +88,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { /** * Estimates partition start indices for post-shuffle partitions based on - * mapOutputStatistics provided by all pre-shuffle fragments. + * mapOutputStatistics provided by all pre-shuffle stages. */ // visible for testing. private[sql] def estimatePartitionStartIndices( @@ -112,7 +112,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " + s"targetPostShuffleInputSize $targetPostShuffleInputSize.") - // Make sure we do get the same number of pre-shuffle partitions for those fragments. + // Make sure we do get the same number of pre-shuffle partitions for those stages. val distinctNumPreShufflePartitions = mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct // The reason that we are expecting a single value of the number of pre-shuffle partitions @@ -135,7 +135,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { var i = 0 while (i < numPreShufflePartitions) { - // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle fragments. + // 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 var j = 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index c5453289d3946..073225ffd2c0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -96,7 +96,7 @@ object SparkPlanGraph { case "InputAdapter" => buildSparkPlanGraphNode( planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) - case "BroadcastQueryFragment" | "ResultQueryFragment" | "ShuffleQueryFragment" => + case "BroadcastQueryStage" | "ShuffleQueryStage" => if (exchanges.contains(planInfo.children.head)) { // Point to the re-used exchange val node = exchanges(planInfo.children.head) From 2e087785d754dfabc84b333fffcf98c39d2fd497 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Fri, 22 Mar 2019 14:21:16 +0800 Subject: [PATCH 21/21] address comments --- .../apache/spark/sql/internal/SQLConf.scala | 14 ++++--- .../execution/adaptive/QueryStageExec.scala | 2 +- .../adaptive/QueryStageManager.scala | 2 +- .../ReduceNumShufflePartitionsSuite.scala | 42 +++++++++---------- 4 files changed, 31 insertions(+), 29 deletions(-) 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 34ddb34237250..a8b632d4e74b0 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 @@ -278,23 +278,24 @@ object SQLConf { val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled") .doc("When true, enable adaptive query execution.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS = buildConf("spark.sql.adaptive.minNumPostShufflePartitions") .doc("The advisory minimum number of post-shuffle partitions used in adaptive execution.") .intConf - .checkValue(numPartitions => numPartitions > 0, "The minimum shuffle partition number " + + .checkValue(_ > 0, "The minimum shuffle partition number " + "must be a positive integer.") .createWithDefault(1) val SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS = buildConf("spark.sql.adaptive.maxNumPostShufflePartitions") - .doc("The advisory maximum number of post-shuffle partitions used in adaptive execution.") + .doc("The advisory maximum number of post-shuffle partitions used in adaptive execution. " + + "The by default equals to spark.sql.shuffle.partitions") .intConf - .checkValue(numPartitions => numPartitions > 0, "The maximum shuffle partition number " + + .checkValue(_ > 0, "The maximum shuffle partition number " + "must be a positive integer.") - .createWithDefault(500) + .createOptional val SUBEXPRESSION_ELIMINATION_ENABLED = buildConf("spark.sql.subexpressionElimination.enabled") @@ -1735,7 +1736,8 @@ class SQLConf extends Serializable with Logging { def minNumPostShufflePartitions: Int = getConf(SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS) - def maxNumPostShufflePartitions: Int = getConf(SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS) + def maxNumPostShufflePartitions: Int = + getConf(SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS).getOrElse(numShufflePartitions) def minBatchesToRetain: Int = getConf(MIN_BATCHES_TO_RETAIN) 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 80c623ea94400..e3a7165fa37a0 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 @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution.exchange._ * There are 2 kinds of query stages: * 1. Shuffle query stage. This stage materializes its output to shuffle files, and Spark launches * another job to execute the further operators. - * 2. Broadcast stage. This stage materializes its output to an array in driver JVM. Spark + * 2. Broadcast query stage. This stage materializes its output to an array in driver JVM. Spark * broadcasts the array before executing the further operators. */ abstract class QueryStageExec extends LeafExecNode { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala index a98403b6727f2..0c4f093869a81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageManager.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.{EventLoop, ThreadUtils} * * When one query stage finishes materialization, a list of adaptive optimizer rules will be * executed, trying to optimize the query plan with the data statistics collected from the the - * materialized data. Then we travers the query plan again and try to insert more query stages. + * materialized data. Then we traverse the query plan again and try to insert more query stages. * * To create query stages, we traverse the query tree bottom up. When we hit an exchange node, * and all the child query stages of this exchange node are materialized, we create a new 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 83deae2342a37..dd4b5591ad8e9 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 @@ -527,11 +527,11 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA spark.sql("SET spark.sql.exchange.reuse=true") val df = spark.range(1).selectExpr("id AS key", "id AS value") - // test case 1: a fragment has 3 child fragments but they are the same fragment. - // ResultQueryFragment 1 - // ShuffleQueryFragment 0 - // ReusedQueryFragment 0 - // ReusedQueryFragment 0 + // test case 1: a query stage has 3 child stages but they are the same stage. + // ResultQueryStage 1 + // ShuffleQueryStage 0 + // ReusedQueryStage 0 + // ReusedQueryStage 0 val resultDf = df.join(df, "key").join(df, "key") val finalPlan = resultDf.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].finalPlan @@ -539,12 +539,12 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 3) checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) - // test case 2: a fragment has 2 parent fragments. - // ResultQueryFragment 3 - // ShuffleQueryFragment 1 - // ShuffleQueryFragment 0 - // ShuffleQueryFragment 2 - // ReusedQueryFragment 0 + // test case 2: a query stage has 2 parent stages. + // ResultQueryStage 3 + // ShuffleQueryStage 1 + // ShuffleQueryStage 0 + // ShuffleQueryStage 2 + // ReusedQueryStage 0 val grouped = df.groupBy("key").agg(max("value").as("value")) val resultDf2 = grouped.groupBy(col("key") + 1).max("value") .union(grouped.groupBy(col("key") + 2).max("value")) @@ -552,22 +552,22 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val finalPlan2 = resultDf2.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].finalPlan - // The result fragment has 2 children - val level1Fragments = finalPlan2.collect { case q: QueryStageExec => q } - assert(level1Fragments.length == 2) + // The result stage has 2 children + val level1Stages = finalPlan2.collect { case q: QueryStageExec => q } + assert(level1Stages.length == 2) - val leafFragments = level1Fragments.flatMap { fragment => - // All of the child fragments of result fragment have only one child fragment. - val children = fragment.plan.collect { case q: QueryStageExec => q } + val leafStages = level1Stages.flatMap { stage => + // All of the child stages of result stage have only one child stage. + val children = stage.plan.collect { case q: QueryStageExec => q } assert(children.length == 1) children } - assert(leafFragments.length == 2) + assert(leafStages.length == 2) - val reusedFragments = level1Fragments.flatMap { fragment => - fragment.plan.collect { case r: ReusedQueryStageExec => r } + val reusedStages = level1Stages.flatMap { stage => + stage.plan.collect { case r: ReusedQueryStageExec => r } } - assert(reusedFragments.length == 1) + assert(reusedStages.length == 1) checkAnswer(resultDf2, Row(1, 0) :: Row(2, 0) :: Nil) }