[GitHub] [spark] gatorsmile commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
gatorsmile 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_r293266268 ## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala ## @@ -0,0 +1,446 @@ +/* + * 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 + + @volatile private var fallback = 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
[GitHub] [spark] gatorsmile commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
gatorsmile 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_r292898739 ## File path: sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala ## @@ -0,0 +1,275 @@ +/* + * 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 org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { + + setupTestData() + + private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { +val dfAdaptive = sql(query) +val result = dfAdaptive.collect() +withSQLConf(SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "false") { + val df = sql(query) + QueryTest.sameRows(result.toSeq, df.collect().toSeq) +} +val plan = dfAdaptive.queryExecution.executedPlan +assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) +val adaptivePlan = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan +(dfAdaptive.queryExecution.sparkPlan, adaptivePlan) + } + + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { +plan.collect { + case j: BroadcastHashJoinExec => Seq(j) + case s: QueryStageExec => findTopLevelBroadcastHashJoin(s.plan) +}.flatten + } + + private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { +plan.collect { + case j: SortMergeJoinExec => Seq(j) + case s: QueryStageExec => findTopLevelSortMergeJoin(s.plan) +}.flatten + } + + private def findReusedExchange(plan: SparkPlan): Seq[ReusedQueryStageExec] = { +plan.collect { + case e: ReusedQueryStageExec => Seq(e) + case a: AdaptiveSparkPlanExec => findReusedExchange(a.executedPlan) + case s: QueryStageExec => findReusedExchange(s.plan) + case p: SparkPlan => p.subqueries.flatMap(findReusedExchange) +}.flatten + } + + private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { +plan.collect { + case e: ReusedSubqueryExec => Seq(e) + case s: QueryStageExec => findReusedSubquery(s.plan) + case p: SparkPlan => p.subqueries.flatMap(findReusedSubquery) +}.flatten + } + + test("Change merge join to broadcast join") { +withSQLConf( +SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", +SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( +"SELECT * FROM testData join testData2 ON key = a where value = '1'") Review comment: ```Scala withSQLConf( SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { spark.sql( "SELECT * FROM testData join testData2 ON key = a where value = '1'").explain() } withSQLConf( SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { spark.sql( "SELECT * FROM testData join testData2 ON key = a where value = '1'").explain() } ``` Try these? It sounds like the WholeStage codegen ID is lost after we turn on the adaptive query execution. 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
[GitHub] [spark] gatorsmile commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
gatorsmile 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_r292904576 ## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala ## @@ -0,0 +1,446 @@ +/* + * 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 + + @volatile private var fallback = 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
[GitHub] [spark] gatorsmile commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
gatorsmile 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_r292904576 ## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala ## @@ -0,0 +1,446 @@ +/* + * 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 + + @volatile private var fallback = 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
[GitHub] [spark] gatorsmile commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
gatorsmile 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_r292904576 ## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala ## @@ -0,0 +1,446 @@ +/* + * 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 + + @volatile private var fallback = 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
[GitHub] [spark] gatorsmile commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
gatorsmile 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_r292900640 ## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala ## @@ -288,6 +288,27 @@ class SQLAppStatusListener( update(exec) } + private def onAdaptiveExecutionUpdate(event: SparkListenerSQLAdaptiveExecutionUpdate): Unit = { +val SparkListenerSQLAdaptiveExecutionUpdate(executionId, +physicalPlanDescription, sparkPlanInfo) = event Review comment: Nit: style ``` val SparkListenerSQLAdaptiveExecutionUpdate( executionId, physicalPlanDescription, sparkPlanInfo) = event ``` 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
[GitHub] [spark] gatorsmile commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
gatorsmile 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_r292899611 ## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala ## @@ -248,26 +248,26 @@ class SQLAppStatusListener( } } + def toStoredNodes(nodes: Seq[SparkPlanGraphNode]): Seq[SparkPlanGraphNodeWrapper] = { Review comment: Nit: `def` -> `private def`? 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
[GitHub] [spark] gatorsmile commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
gatorsmile 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_r292898739 ## File path: sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala ## @@ -0,0 +1,275 @@ +/* + * 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 org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { + + setupTestData() + + private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { +val dfAdaptive = sql(query) +val result = dfAdaptive.collect() +withSQLConf(SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "false") { + val df = sql(query) + QueryTest.sameRows(result.toSeq, df.collect().toSeq) +} +val plan = dfAdaptive.queryExecution.executedPlan +assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) +val adaptivePlan = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan +(dfAdaptive.queryExecution.sparkPlan, adaptivePlan) + } + + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { +plan.collect { + case j: BroadcastHashJoinExec => Seq(j) + case s: QueryStageExec => findTopLevelBroadcastHashJoin(s.plan) +}.flatten + } + + private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { +plan.collect { + case j: SortMergeJoinExec => Seq(j) + case s: QueryStageExec => findTopLevelSortMergeJoin(s.plan) +}.flatten + } + + private def findReusedExchange(plan: SparkPlan): Seq[ReusedQueryStageExec] = { +plan.collect { + case e: ReusedQueryStageExec => Seq(e) + case a: AdaptiveSparkPlanExec => findReusedExchange(a.executedPlan) + case s: QueryStageExec => findReusedExchange(s.plan) + case p: SparkPlan => p.subqueries.flatMap(findReusedExchange) +}.flatten + } + + private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { +plan.collect { + case e: ReusedSubqueryExec => Seq(e) + case s: QueryStageExec => findReusedSubquery(s.plan) + case p: SparkPlan => p.subqueries.flatMap(findReusedSubquery) +}.flatten + } + + test("Change merge join to broadcast join") { +withSQLConf( +SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", +SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( +"SELECT * FROM testData join testData2 ON key = a where value = '1'") Review comment: ```Scala withSQLConf( SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { spark.sql( "SELECT * FROM testData join testData2 ON key = a where value = '1'").explain() } withSQLConf( SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { spark.sql( "SELECT * FROM testData join testData2 ON key = a where value = '1'").explain() } ``` Try these? It sounds like we are unable to tell the difference from the physical plans. Also, the WholeStage codegen ID is lost after we turn on the adaptive query execution. 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: