hvanhovell commented on a change in pull request #24706: [SPARK-23128][SQL] A 
new approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#discussion_r293122322
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
 ##########
 @@ -0,0 +1,410 @@
+/*
+ * 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
+import java.util.concurrent.LinkedBlockingQueue
+
+import scala.collection.JavaConverters._
+import scala.collection.concurrent.TrieMap
+import scala.collection.mutable
+import scala.concurrent.ExecutionContext
+
+import org.apache.spark.SparkException
+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.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec._
+import org.apache.spark.sql.execution.exchange._
+import 
org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * 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 the
+ * materialized output will be used to optimize the remainder of the query.
+ *
+ * To create query stages, we traverse the query tree bottom up. When we hit 
an exchange node,
+ * and if all the child query stages of this exchange node are materialized, 
we create a new
+ * query stage for this exchange node. The new stage is then materialized 
asynchronously once it
+ * is created.
+ *
+ * When one query stage finishes materialization, the rest query is 
re-optimized and planned based
+ * on the latest statistics provided by all materialized stages. Then we 
traverse the query plan
+ * again and create more stages if possible. After all stages have been 
materialized, we execute
+ * the rest of the plan.
+ */
+case class AdaptiveSparkPlanExec(
+    initialPlan: SparkPlan,
+    @transient session: SparkSession,
+    @transient subqueryMap: Map[Long, ExecSubqueryExpression],
+    @transient stageCache: TrieMap[SparkPlan, QueryStageExec])
+  extends LeafExecNode {
+
+  @transient private val executionId = Option(
+    
session.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).map(_.toLong)
+
+  @transient private val lock = new Object()
+
+  // The logical plan optimizer for re-optimizing the current logical plan.
+  @transient private val optimizer = new RuleExecutor[LogicalPlan] {
+    // TODO add more optimization rules
+    override protected def batches: Seq[Batch] = Seq()
+  }
+
+  // A list of physical plan rules to be applied before creation of query 
stages. The physical
+  // plan should reach a final status of query stages (i.e., no more addition 
or removal of
+  // Exchange nodes) after running these rules.
+  @transient private val queryStagePreparationRules: Seq[Rule[SparkPlan]] = 
Seq(
+    PlanAdaptiveSubqueries(subqueryMap),
+    EnsureRequirements(conf)
+  )
+
+  // A list of physical optimizer rules to be applied to a new stage before 
its execution. These
+  // optimizations should be stage-independent.
+  @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq(
+    CollapseCodegenStages(conf)
+  )
+
+  private var currentStageId = 0
+
+  @volatile private var currentPhysicalPlan = initialPlan
+
+  @volatile private var isFinalPlan = false
+
+  /**
+   * Return type for `createQueryStages`
+   * @param newPlan the new plan with created query stages.
+   * @param allChildStagesMaterialized whether all child stages have been 
materialized.
+   * @param newStages the newly created query stages, including new reused 
query stages.
+   */
+  private case class CreateStageResult(
+    newPlan: SparkPlan,
+    allChildStagesMaterialized: Boolean,
+    newStages: Seq[(Exchange, QueryStageExec)])
+
+  def executedPlan: SparkPlan = currentPhysicalPlan
+
+  override def conf: SQLConf = session.sessionState.conf
+
+  override def output: Seq[Attribute] = initialPlan.output
+
+  override def doCanonicalize(): SparkPlan = initialPlan.canonicalized
+
+  override def doExecute(): RDD[InternalRow] = if (isFinalPlan) {
+    currentPhysicalPlan.execute()
+  } else {
+    lock.synchronized {
+      var currentLogicalPlan = currentPhysicalPlan.logicalLink.get
+      var result = createQueryStages(currentPhysicalPlan)
+      val events = new LinkedBlockingQueue[StageMaterializationEvent]()
+      val errors = new mutable.ArrayBuffer[SparkException]()
+      while (!result.allChildStagesMaterialized) {
+        currentPhysicalPlan = result.newPlan
+        currentLogicalPlan = updateLogicalPlan(currentLogicalPlan, 
result.newStages)
+        currentPhysicalPlan.setTagValue(SparkPlan.LOGICAL_PLAN_TAG, 
currentLogicalPlan)
+        onUpdatePlan()
+
+        // Start materialization of all new stages.
+        result.newStages.map(_._2).foreach { stage =>
+          stage.materialize().onComplete { res =>
+            if (res.isSuccess) {
+              events.offer(StageSuccess(stage, res.get))
+            } else {
+              events.offer(StageFailure(stage, res.failed.get))
+            }
+          }(AdaptiveSparkPlanExec.executionContext)
+        }
+
+        // Wait on the next completed stage, which indicates new stats are 
available and probably
+        // new stages can be created. There might be other stages that finish 
at around the same
+        // time, so we process those stages too in order to reduce re-planning.
+        val nextMsg = events.take()
+        val rem = new util.ArrayList[StageMaterializationEvent]()
+        events.drainTo(rem)
+        (Seq(nextMsg) ++ rem.asScala).foreach {
+          case StageSuccess(stage, res) =>
+            stage.resultOption = Some(res)
+          case StageFailure(stage, ex) =>
+            errors.append(
+              new SparkException(s"Fail to materialize query stage: 
${stage.treeString}", ex))
+        }
+
+        // In case of errors, we cancel all running stages and throw exception.
+        if (errors.nonEmpty) {
+          try {
+            currentPhysicalPlan.foreach {
+              case s: QueryStageExec => s.cancel()
 
 Review comment:
   Can we make cancellation best effort? Now we stop cancellation as soon as we 
hit the first error. It would also be good if we'd add the cancellation errors 
to the errors buffer.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to