|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.execution.adaptive |
| 19 | + |
| 20 | +import scala.concurrent.{ExecutionContext, Future} |
| 21 | +import scala.concurrent.duration.Duration |
| 22 | + |
| 23 | +import org.apache.spark.MapOutputStatistics |
| 24 | +import org.apache.spark.broadcast |
| 25 | +import org.apache.spark.rdd.RDD |
| 26 | +import org.apache.spark.sql.catalyst.InternalRow |
| 27 | +import org.apache.spark.sql.catalyst.expressions._ |
| 28 | +import org.apache.spark.sql.catalyst.plans.physical.Partitioning |
| 29 | +import org.apache.spark.sql.execution._ |
| 30 | +import org.apache.spark.sql.execution.exchange._ |
| 31 | +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate |
| 32 | +import org.apache.spark.util.ThreadUtils |
| 33 | + |
| 34 | +/** |
| 35 | + * In adaptive execution mode, an execution plan is divided into multiple QueryStages. Each |
| 36 | + * QueryStage is a sub-tree that runs in a single stage. |
| 37 | + */ |
| 38 | +abstract class QueryStage extends UnaryExecNode { |
| 39 | + |
| 40 | + var child: SparkPlan |
| 41 | + |
| 42 | + // Ignore this wrapper for canonicalizing. |
| 43 | + override def doCanonicalize(): SparkPlan = child.canonicalized |
| 44 | + |
| 45 | + override def output: Seq[Attribute] = child.output |
| 46 | + |
| 47 | + override def outputPartitioning: Partitioning = child.outputPartitioning |
| 48 | + |
| 49 | + override def outputOrdering: Seq[SortOrder] = child.outputOrdering |
| 50 | + |
| 51 | + /** |
| 52 | + * Execute childStages and wait until all stages are completed. Use a thread pool to avoid |
| 53 | + * blocking on one child stage. |
| 54 | + */ |
| 55 | + def executeChildStages(): Unit = { |
| 56 | + // Handle broadcast stages |
| 57 | + val broadcastQueryStages: Seq[BroadcastQueryStage] = child.collect { |
| 58 | + case bqs: BroadcastQueryStageInput => bqs.childStage |
| 59 | + } |
| 60 | + val broadcastFutures = broadcastQueryStages.map { queryStage => |
| 61 | + Future { queryStage.prepareBroadcast() }(QueryStage.executionContext) |
| 62 | + } |
| 63 | + |
| 64 | + // Submit shuffle stages |
| 65 | + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) |
| 66 | + val shuffleQueryStages: Seq[ShuffleQueryStage] = child.collect { |
| 67 | + case sqs: ShuffleQueryStageInput => sqs.childStage |
| 68 | + } |
| 69 | + val shuffleStageFutures = shuffleQueryStages.map { queryStage => |
| 70 | + Future { |
| 71 | + SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) { |
| 72 | + queryStage.execute() |
| 73 | + } |
| 74 | + }(QueryStage.executionContext) |
| 75 | + } |
| 76 | + |
| 77 | + ThreadUtils.awaitResult( |
| 78 | + Future.sequence(broadcastFutures)(implicitly, QueryStage.executionContext), Duration.Inf) |
| 79 | + ThreadUtils.awaitResult( |
| 80 | + Future.sequence(shuffleStageFutures)(implicitly, QueryStage.executionContext), Duration.Inf) |
| 81 | + } |
| 82 | + |
| 83 | + /** |
| 84 | + * Before executing the plan in this query stage, we execute all child stages, optimize the plan |
| 85 | + * in this stage and determine the reducer number based on the child stages' statistics. Finally |
| 86 | + * we do a codegen for this query stage and update the UI with the new plan. |
| 87 | + */ |
| 88 | + def prepareExecuteStage(): Unit = { |
| 89 | + // 1. Execute childStages |
| 90 | + executeChildStages() |
| 91 | + // It is possible to optimize this stage's plan here based on the child stages' statistics. |
| 92 | + |
| 93 | + // 2. Determine reducer number |
| 94 | + val queryStageInputs: Seq[ShuffleQueryStageInput] = child.collect { |
| 95 | + case input: ShuffleQueryStageInput => input |
| 96 | + } |
| 97 | + val childMapOutputStatistics = queryStageInputs.map(_.childStage.mapOutputStatistics) |
| 98 | + .filter(_ != null).toArray |
| 99 | + if (childMapOutputStatistics.length > 0) { |
| 100 | + val exchangeCoordinator = new ExchangeCoordinator( |
| 101 | + conf.targetPostShuffleInputSize, |
| 102 | + conf.minNumPostShufflePartitions) |
| 103 | + |
| 104 | + val partitionStartIndices = |
| 105 | + exchangeCoordinator.estimatePartitionStartIndices(childMapOutputStatistics) |
| 106 | + child = child.transform { |
| 107 | + case ShuffleQueryStageInput(childStage, output, _) => |
| 108 | + ShuffleQueryStageInput(childStage, output, Some(partitionStartIndices)) |
| 109 | + } |
| 110 | + } |
| 111 | + |
| 112 | + // 3. Codegen and update the UI |
| 113 | + child = CollapseCodegenStages(sqlContext.conf).apply(child) |
| 114 | + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) |
| 115 | + if (executionId != null && executionId.nonEmpty) { |
| 116 | + val queryExecution = SQLExecution.getQueryExecution(executionId.toLong) |
| 117 | + sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( |
| 118 | + executionId.toLong, |
| 119 | + queryExecution.toString, |
| 120 | + SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan))) |
| 121 | + } |
| 122 | + } |
| 123 | + |
| 124 | + // Caches the created ShuffleRowRDD so we can reuse that. |
| 125 | + private var cachedRDD: RDD[InternalRow] = null |
| 126 | + |
| 127 | + def executeStage(): RDD[InternalRow] = child.execute() |
| 128 | + |
| 129 | + /** |
| 130 | + * A QueryStage can be reused like Exchange. It is possible that multiple threads try to submit |
| 131 | + * the same QueryStage. Use synchronized to make sure it is executed only once. |
| 132 | + */ |
| 133 | + override def doExecute(): RDD[InternalRow] = synchronized { |
| 134 | + if (cachedRDD == null) { |
| 135 | + prepareExecuteStage() |
| 136 | + cachedRDD = executeStage() |
| 137 | + } |
| 138 | + cachedRDD |
| 139 | + } |
| 140 | + |
| 141 | + override def executeCollect(): Array[InternalRow] = { |
| 142 | + prepareExecuteStage() |
| 143 | + child.executeCollect() |
| 144 | + } |
| 145 | + |
| 146 | + override def executeToIterator(): Iterator[InternalRow] = { |
| 147 | + prepareExecuteStage() |
| 148 | + child.executeToIterator() |
| 149 | + } |
| 150 | + |
| 151 | + override def executeTake(n: Int): Array[InternalRow] = { |
| 152 | + prepareExecuteStage() |
| 153 | + child.executeTake(n) |
| 154 | + } |
| 155 | + |
| 156 | + override def generateTreeString( |
| 157 | + depth: Int, |
| 158 | + lastChildren: Seq[Boolean], |
| 159 | + builder: StringBuilder, |
| 160 | + verbose: Boolean, |
| 161 | + prefix: String = "", |
| 162 | + addSuffix: Boolean = false): StringBuilder = { |
| 163 | + child.generateTreeString(depth, lastChildren, builder, verbose, "*") |
| 164 | + } |
| 165 | +} |
| 166 | + |
| 167 | +/** |
| 168 | + * The last QueryStage of an execution plan. |
| 169 | + */ |
| 170 | +case class ResultQueryStage(var child: SparkPlan) extends QueryStage |
| 171 | + |
| 172 | +/** |
| 173 | + * A shuffle QueryStage whose child is a ShuffleExchange. |
| 174 | + */ |
| 175 | +case class ShuffleQueryStage(var child: SparkPlan) extends QueryStage { |
| 176 | + |
| 177 | + protected var _mapOutputStatistics: MapOutputStatistics = null |
| 178 | + |
| 179 | + def mapOutputStatistics: MapOutputStatistics = _mapOutputStatistics |
| 180 | + |
| 181 | + override def executeStage(): RDD[InternalRow] = { |
| 182 | + child match { |
| 183 | + case e: ShuffleExchangeExec => |
| 184 | + val result = e.eagerExecute() |
| 185 | + _mapOutputStatistics = e.mapOutputStatistics |
| 186 | + result |
| 187 | + case _ => throw new IllegalArgumentException( |
| 188 | + "The child of ShuffleQueryStage must be a ShuffleExchange.") |
| 189 | + } |
| 190 | + } |
| 191 | +} |
| 192 | + |
| 193 | +/** |
| 194 | + * A broadcast QueryStage whose child is a BroadcastExchangeExec. |
| 195 | + */ |
| 196 | +case class BroadcastQueryStage(var child: SparkPlan) extends QueryStage { |
| 197 | + override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { |
| 198 | + child.executeBroadcast() |
| 199 | + } |
| 200 | + |
| 201 | + private var prepared = false |
| 202 | + |
| 203 | + def prepareBroadcast() : Unit = synchronized { |
| 204 | + if (!prepared) { |
| 205 | + executeChildStages() |
| 206 | + child = CollapseCodegenStages(sqlContext.conf).apply(child) |
| 207 | + // After child stages are completed, prepare() triggers the broadcast. |
| 208 | + prepare() |
| 209 | + prepared = true |
| 210 | + } |
| 211 | + } |
| 212 | + |
| 213 | + override def doExecute(): RDD[InternalRow] = { |
| 214 | + throw new UnsupportedOperationException( |
| 215 | + "BroadcastExchange does not support the execute() code path.") |
| 216 | + } |
| 217 | +} |
| 218 | + |
| 219 | +object QueryStage { |
| 220 | + private[execution] val executionContext = ExecutionContext.fromExecutorService( |
| 221 | + ThreadUtils.newDaemonCachedThreadPool("adaptive-query-stage")) |
| 222 | +} |
0 commit comments