maryannxue 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_r288309210
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala
 ##########
 @@ -0,0 +1,118 @@
+/*
+ * 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.concurrent.TrieMap
+import scala.collection.mutable
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.command.ExecutedCommandExec
+import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * This rule wraps the query plan with an [[AdaptiveSparkPlanExec]], which 
executes the query plan
+ * and re-optimize the plan during execution based on runtime data statistics.
+ */
+case class InsertAdaptiveSparkPlan(session: SparkSession) extends 
Rule[SparkPlan] {
+
+  private val conf = session.sessionState.conf
+
+  // Exchange-reuse is shared across the entire query, including sub-queries.
+  private val stageCache = new TrieMap[StructType, mutable.Buffer[(Exchange, 
QueryStageExec)]]()
+
+  override def apply(plan: SparkPlan): SparkPlan = plan match {
+    case _: ExecutedCommandExec => plan
+    case _ if conf.runtimeReoptimizationEnabled
+      && supportAdaptive(plan) =>
+      try {
+        // Plan sub-queries recursively and pass in the shared stage cache for 
exchange reuse. Fall
+        // back to non-adaptive mode if adaptive execution is supported in any 
of the sub-queries.
+        val subqueryMap = planSubqueries(plan)
+        // Run preparation rules.
+        val preparations = 
AdaptiveSparkPlanExec.createQueryStagePreparationRules(
+          session.sessionState.conf, subqueryMap)
+        val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, 
preparations)
 
 Review comment:
   > This already seems to be done in AdaptiveSparkPlanExec when we submit the 
stage.
   
   No.
   
   The physical transformations we used in `QueryExecution.preparations` have 
now been split into two groups in adaptive execution here (also noted in the 
code comment):
   1. Rules that add or remove exchanges.
   2. Rules that are independent within each exchange, or say, stage.
   
   `InsertAdaptiveSparkPlan` is now the first in `QueryExecution.preparations`, 
which means neither of these two groups has been applied yet. It is this way so 
that we do not need to manipulate (modify) the rule application order in 
`QueryExecution.preparations` for AQE.

----------------------------------------------------------------
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