|
| 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 java.util.concurrent.CountDownLatch |
| 21 | + |
| 22 | +import org.apache.spark.rdd.RDD |
| 23 | +import org.apache.spark.sql.SparkSession |
| 24 | +import org.apache.spark.sql.catalyst.InternalRow |
| 25 | +import org.apache.spark.sql.catalyst.expressions.Attribute |
| 26 | +import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, SparkPlanInfo, SQLExecution} |
| 27 | +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate |
| 28 | + |
| 29 | +/** |
| 30 | + * A root node to trigger query stages and execute the query plan adaptively. It incrementally |
| 31 | + * updates the query plan when a query stage is materialized and provides accurate runtime |
| 32 | + * statistics. |
| 33 | + */ |
| 34 | +case class AdaptiveSparkPlan(initialPlan: ResultQueryStage, session: SparkSession) |
| 35 | + extends LeafExecNode{ |
| 36 | + |
| 37 | + override def output: Seq[Attribute] = initialPlan.output |
| 38 | + |
| 39 | + @volatile private var currentQueryStage: QueryStage = initialPlan |
| 40 | + @volatile private var error: Throwable = null |
| 41 | + private val readyLock = new CountDownLatch(1) |
| 42 | + |
| 43 | + private def replaceStage(oldStage: QueryStage, newStage: QueryStage): QueryStage = { |
| 44 | + if (oldStage.id == newStage.id) { |
| 45 | + newStage |
| 46 | + } else { |
| 47 | + val newPlanForOldStage = oldStage.plan.transform { |
| 48 | + case q: QueryStage => replaceStage(q, newStage) |
| 49 | + } |
| 50 | + oldStage.withNewPlan(newPlanForOldStage) |
| 51 | + } |
| 52 | + } |
| 53 | + |
| 54 | + private def createCallback(executionId: Option[Long]): QueryStageTriggerCallback = { |
| 55 | + new QueryStageTriggerCallback { |
| 56 | + override def onStageUpdated(stage: QueryStage): Unit = { |
| 57 | + updateCurrentQueryStage(stage, executionId) |
| 58 | + if (stage.isInstanceOf[ResultQueryStage]) readyLock.countDown() |
| 59 | + } |
| 60 | + |
| 61 | + override def onStagePlanningFailed(stage: QueryStage, e: Throwable): Unit = { |
| 62 | + error = new RuntimeException( |
| 63 | + s""" |
| 64 | + |Fail to plan stage ${stage.id}: |
| 65 | + |${stage.plan.treeString} |
| 66 | + """.stripMargin, e) |
| 67 | + readyLock.countDown() |
| 68 | + } |
| 69 | + |
| 70 | + override def onStageMaterializingFailed(stage: QueryStage, e: Throwable): Unit = { |
| 71 | + error = new RuntimeException( |
| 72 | + s""" |
| 73 | + |Fail to materialize stage ${stage.id}: |
| 74 | + |${stage.plan.treeString} |
| 75 | + """.stripMargin, e) |
| 76 | + readyLock.countDown() |
| 77 | + } |
| 78 | + |
| 79 | + override def onError(e: Throwable): Unit = { |
| 80 | + error = e |
| 81 | + readyLock.countDown() |
| 82 | + } |
| 83 | + } |
| 84 | + } |
| 85 | + |
| 86 | + private def updateCurrentQueryStage(newStage: QueryStage, executionId: Option[Long]): Unit = { |
| 87 | + currentQueryStage = replaceStage(currentQueryStage, newStage) |
| 88 | + executionId.foreach { id => |
| 89 | + session.sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( |
| 90 | + id, |
| 91 | + SQLExecution.getQueryExecution(id).toString, |
| 92 | + SparkPlanInfo.fromSparkPlan(currentQueryStage))) |
| 93 | + } |
| 94 | + } |
| 95 | + |
| 96 | + def resultStage: ResultQueryStage = { |
| 97 | + if (readyLock.getCount > 0) { |
| 98 | + val sc = session.sparkContext |
| 99 | + val executionId = Option(sc.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).map(_.toLong) |
| 100 | + val trigger = new QueryStageTrigger(session, createCallback(executionId)) |
| 101 | + trigger.start() |
| 102 | + trigger.trigger(initialPlan) |
| 103 | + readyLock.await() |
| 104 | + trigger.stop() |
| 105 | + } |
| 106 | + |
| 107 | + if (error != null) throw error |
| 108 | + currentQueryStage.asInstanceOf[ResultQueryStage] |
| 109 | + } |
| 110 | + |
| 111 | + override def executeCollect(): Array[InternalRow] = resultStage.executeCollect() |
| 112 | + override def executeTake(n: Int): Array[InternalRow] = resultStage.executeTake(n) |
| 113 | + override def executeToIterator(): Iterator[InternalRow] = resultStage.executeToIterator() |
| 114 | + override def doExecute(): RDD[InternalRow] = resultStage.execute() |
| 115 | + override def generateTreeString( |
| 116 | + depth: Int, |
| 117 | + lastChildren: Seq[Boolean], |
| 118 | + append: String => Unit, |
| 119 | + verbose: Boolean, |
| 120 | + prefix: String = "", |
| 121 | + addSuffix: Boolean = false, |
| 122 | + maxFields: Int): Unit = { |
| 123 | + currentQueryStage.generateTreeString( |
| 124 | + depth, lastChildren, append, verbose, "", false, maxFields) |
| 125 | + } |
| 126 | +} |
0 commit comments