@@ -28,9 +28,11 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, SparkPlanInfo, S
2828import org .apache .spark .sql .execution .ui .SparkListenerSQLAdaptiveExecutionUpdate
2929
3030/**
31- * A root node to execute the query plan adaptively. It creates query fragments, and incrementally
32- * updates the query plan when a query fragment is materialized and provides accurate runtime
33- * data statistics.
31+ * A root node to execute the query plan adaptively. It splits the query plan into independent
32+ * stages and executes them in order according to their dependencies. The query stage
33+ * materializes its output at the end. When one stage completes, the data statistics of its
34+ * materialized output will be used to optimize the subsequent stages.
35+ * This is called mid-query re-optimization in database literature.
3436 */
3537case class AdaptiveSparkPlanExec (initialPlan : SparkPlan , session : SparkSession )
3638 extends LeafExecNode {
@@ -40,34 +42,34 @@ case class AdaptiveSparkPlanExec(initialPlan: SparkPlan, session: SparkSession)
4042 @ volatile private var currentPlan : SparkPlan = initialPlan
4143 @ volatile private var error : Throwable = null
4244
43- // We will release the lock when we finish planning query fragments , or we fail to do the
44- // planning . Getting `finalPlan` will be blocked until the lock is release.
45+ // We will release the lock when all the query stages are completed , or we fail to
46+ // optimize/execute query stages . Getting `finalPlan` will be blocked until the lock is release.
4547 // This is better than wait()/notify(), as we can easily check if the computation has completed,
4648 // by calling `readyLock.getCount()`.
4749 private val readyLock = new CountDownLatch (1 )
4850
49- private def createCallback (executionId : Option [Long ]): QueryFragmentCreatorCallback = {
50- new QueryFragmentCreatorCallback {
51- override def onPlanUpdate (updatedPlan : SparkPlan ): Unit = {
52- updateCurrentPlan(updatedPlan, executionId)
53- if (updatedPlan.isInstanceOf [ResultQueryFragmentExec ]) readyLock.countDown()
54- }
51+ private def createCallback (executionId : Option [Long ]) = new QueryStageManagerCallback {
52+ override def onPlanUpdate (updatedPlan : SparkPlan ): Unit = {
53+ updateCurrentPlan(updatedPlan, executionId)
54+ }
55+
56+ override def onFinalPlan (finalPlan : SparkPlan ): Unit = {
57+ updateCurrentPlan(finalPlan, executionId)
58+ readyLock.countDown()
59+ }
5560
56- override def onFragmentMaterializingFailed (
57- fragment : QueryFragmentExec ,
58- e : Throwable ): Unit = {
59- error = new SparkException (
60- s """
61- |Fail to materialize fragment ${fragment.id}:
62- | ${fragment.plan.treeString}
61+ override def onStageMaterializationFailed (stage : QueryStageExec , e : Throwable ): Unit = {
62+ error = new SparkException (
63+ s """
64+ |Fail to materialize query stage ${stage.id}:
65+ | ${stage.plan.treeString}
6366 """ .stripMargin, e)
64- readyLock.countDown()
65- }
67+ readyLock.countDown()
68+ }
6669
67- override def onError (e : Throwable ): Unit = {
68- error = e
69- readyLock.countDown()
70- }
70+ override def onError (e : Throwable ): Unit = {
71+ error = e
72+ readyLock.countDown()
7173 }
7274 }
7375
@@ -81,18 +83,18 @@ case class AdaptiveSparkPlanExec(initialPlan: SparkPlan, session: SparkSession)
8183 }
8284 }
8385
84- def finalPlan : ResultQueryFragmentExec = {
86+ def finalPlan : SparkPlan = {
8587 if (readyLock.getCount > 0 ) {
8688 val sc = session.sparkContext
8789 val executionId = Option (sc.getLocalProperty(SQLExecution .EXECUTION_ID_KEY )).map(_.toLong)
88- val creator = new QueryFragmentCreator (initialPlan, session, createCallback(executionId))
89- creator .start()
90+ val stageManager = new QueryStageManager (initialPlan, session, createCallback(executionId))
91+ stageManager .start()
9092 readyLock.await()
91- creator .stop()
93+ stageManager .stop()
9294 }
9395
9496 if (error != null ) throw error
95- currentPlan. asInstanceOf [ ResultQueryFragmentExec ]
97+ currentPlan
9698 }
9799
98100 override def executeCollect (): Array [InternalRow ] = finalPlan.executeCollect()
0 commit comments