-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-42101][SQL] Make AQE support InMemoryTableScanExec #39624
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
fc4749d
cd42dd5
295e47e
98f7da4
add320a
4b749d8
08e8f80
c9a5040
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -40,6 +40,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors | |
| import org.apache.spark.sql.execution._ | ||
| import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec._ | ||
| import org.apache.spark.sql.execution.bucketing.DisableUnnecessaryBucketedScan | ||
| import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec | ||
| import org.apache.spark.sql.execution.exchange._ | ||
| import org.apache.spark.sql.execution.ui.{SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLAdaptiveSQLMetricUpdates, SQLPlanMetric} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
|
|
@@ -220,10 +221,24 @@ case class AdaptiveSparkPlanExec( | |
| } | ||
|
|
||
| private def getExecutionId: Option[Long] = { | ||
| // If the `QueryExecution` does not match the current execution ID, it means the execution ID | ||
| // belongs to another (parent) query, and we should not call update UI in this query. | ||
| Option(context.session.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)) | ||
| .map(_.toLong).filter(SQLExecution.getQueryExecution(_) eq context.qe) | ||
| .map(_.toLong) | ||
| } | ||
|
|
||
| private lazy val shouldUpdatePlan: Boolean = { | ||
| // There are two cases that should not update plan: | ||
| // 1. When executing subqueries, we can't update the query plan in the UI as the | ||
| // UI doesn't support partial update yet. However, the subquery may have been | ||
| // optimized into a different plan and we must let the UI know the SQL metrics | ||
| // of the new plan nodes, so that it can track the valid accumulator updates later | ||
| // and display SQL metrics correctly. | ||
| // 2. If the `QueryExecution` does not match the current execution ID, it means the execution | ||
| // ID belongs to another (parent) query, and we should not call update UI in this query. | ||
| // e.g., a nested `AdaptiveSparkPlanExec` in `InMemoryTableScanExec`. | ||
| // | ||
| // That means only the root `AdaptiveSparkPlanExec` of the main query that triggers this | ||
| // query execution need to do a plan update for the UI. | ||
| !isSubquery && getExecutionId.exists(SQLExecution.getQueryExecution(_) eq context.qe) | ||
| } | ||
|
|
||
| def finalPhysicalPlan: SparkPlan = withFinalPlanUpdate(identity) | ||
|
|
@@ -345,7 +360,7 @@ case class AdaptiveSparkPlanExec( | |
| // Subqueries that don't belong to any query stage of the main query will execute after the | ||
| // last UI update in `getFinalPhysicalPlan`, so we need to update UI here again to make sure | ||
| // the newly generated nodes of those subqueries are updated. | ||
| if (!isSubquery && currentPhysicalPlan.exists(_.subqueries.nonEmpty)) { | ||
| if (shouldUpdatePlan && currentPhysicalPlan.exists(_.subqueries.nonEmpty)) { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| getExecutionId.foreach(onUpdatePlan(_, Seq.empty)) | ||
| } | ||
| logOnLevel(s"Final plan:\n$currentPhysicalPlan") | ||
|
|
@@ -499,12 +514,13 @@ case class AdaptiveSparkPlanExec( | |
| // Create a query stage only when all the child query stages are ready. | ||
| if (result.allChildStagesMaterialized) { | ||
| var newStage = newQueryStage(newPlan) | ||
| assert(newStage.isInstanceOf[ReusableQueryStageExec]) | ||
| if (conf.exchangeReuseEnabled) { | ||
| // Check the `stageCache` again for reuse. If a match is found, ditch the new stage | ||
| // and reuse the existing stage found in the `stageCache`, otherwise update the | ||
| // `stageCache` with the new stage. | ||
| val queryStage = context.stageCache.getOrElseUpdate( | ||
| newStage.plan.canonicalized, newStage) | ||
| newStage.plan.canonicalized, newStage.asInstanceOf[ReusableQueryStageExec]) | ||
| if (queryStage.ne(newStage)) { | ||
| newStage = reuseQueryStage(queryStage, e) | ||
| } | ||
|
|
@@ -520,6 +536,14 @@ case class AdaptiveSparkPlanExec( | |
| } | ||
| } | ||
|
|
||
| case i: InMemoryTableScanExec => | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. question: if the table cache is already materialized (second access of the cache), do we still need to wrap it with
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| val newStage = newQueryStage(i) | ||
| val isMaterialized = newStage.isMaterialized | ||
| CreateStageResult( | ||
| newPlan = newStage, | ||
cloud-fan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| allChildStagesMaterialized = isMaterialized, | ||
| newStages = if (isMaterialized) Seq.empty else Seq(newStage)) | ||
|
|
||
| case q: QueryStageExec => | ||
| CreateStageResult(newPlan = q, | ||
| allChildStagesMaterialized = q.isMaterialized, newStages = Seq.empty) | ||
|
|
@@ -536,10 +560,10 @@ case class AdaptiveSparkPlanExec( | |
| } | ||
| } | ||
|
|
||
| private def newQueryStage(e: Exchange): QueryStageExec = { | ||
| val optimizedPlan = optimizeQueryStage(e.child, isFinalStage = false) | ||
| val queryStage = e match { | ||
| private def newQueryStage(plan: SparkPlan): QueryStageExec = { | ||
| val queryStage = plan match { | ||
| case s: ShuffleExchangeLike => | ||
| val optimizedPlan = optimizeQueryStage(s.child, isFinalStage = false) | ||
| val newShuffle = applyPhysicalRules( | ||
| s.withNewChildren(Seq(optimizedPlan)), | ||
| postStageCreationRules(outputsColumnar = s.supportsColumnar), | ||
|
|
@@ -550,6 +574,7 @@ case class AdaptiveSparkPlanExec( | |
| } | ||
| ShuffleQueryStageExec(currentStageId, newShuffle, s.canonicalized) | ||
| case b: BroadcastExchangeLike => | ||
| val optimizedPlan = optimizeQueryStage(b.child, isFinalStage = false) | ||
| val newBroadcast = applyPhysicalRules( | ||
| b.withNewChildren(Seq(optimizedPlan)), | ||
| postStageCreationRules(outputsColumnar = b.supportsColumnar), | ||
|
|
@@ -559,13 +584,26 @@ case class AdaptiveSparkPlanExec( | |
| "Custom columnar rules cannot transform broadcast node to something else.") | ||
| } | ||
| BroadcastQueryStageExec(currentStageId, newBroadcast, b.canonicalized) | ||
| case i: InMemoryTableScanExec => | ||
| val newInMemoryTableScan = applyPhysicalRules( | ||
| i, | ||
| postStageCreationRules(outputsColumnar = i.supportsColumnar), | ||
| Some((planChangeLogger, "AQE Post Stage Creation"))) | ||
| if (!newInMemoryTableScan.isInstanceOf[InMemoryTableScanExec]) { | ||
| throw new IllegalStateException("Custom columnar rules cannot transform " + | ||
| "`InMemoryTableScanExec` node to something else.") | ||
| } | ||
| TableCacheQueryStageExec( | ||
| currentStageId, newInMemoryTableScan.asInstanceOf[InMemoryTableScanExec]) | ||
| } | ||
| currentStageId += 1 | ||
| setLogicalLinkForNewQueryStage(queryStage, e) | ||
| setLogicalLinkForNewQueryStage(queryStage, plan) | ||
| queryStage | ||
| } | ||
|
|
||
| private def reuseQueryStage(existing: QueryStageExec, exchange: Exchange): QueryStageExec = { | ||
| private def reuseQueryStage( | ||
| existing: ReusableQueryStageExec, | ||
| exchange: Exchange): QueryStageExec = { | ||
| val queryStage = existing.newReuseInstance(currentStageId, exchange.output) | ||
| currentStageId += 1 | ||
| setLogicalLinkForNewQueryStage(queryStage, exchange) | ||
|
|
@@ -707,12 +745,7 @@ case class AdaptiveSparkPlanExec( | |
| * Notify the listeners of the physical plan change. | ||
| */ | ||
| private def onUpdatePlan(executionId: Long, newSubPlans: Seq[SparkPlan]): Unit = { | ||
| if (isSubquery) { | ||
| // When executing subqueries, we can't update the query plan in the UI as the | ||
| // UI doesn't support partial update yet. However, the subquery may have been | ||
| // optimized into a different plan and we must let the UI know the SQL metrics | ||
| // of the new plan nodes, so that it can track the valid accumulator updates later | ||
| // and display SQL metrics correctly. | ||
| if (!shouldUpdatePlan) { | ||
| val newMetrics = newSubPlans.flatMap { p => | ||
| p.flatMap(_.metrics.values.map(m => SQLPlanMetric(m.name.get, m.id, m.metricType))) | ||
| } | ||
|
|
@@ -814,8 +847,8 @@ case class AdaptiveExecutionContext(session: SparkSession, qe: QueryExecution) { | |
| /** | ||
| * The exchange-reuse map shared across the entire query, including sub-queries. | ||
| */ | ||
| val stageCache: TrieMap[SparkPlan, QueryStageExec] = | ||
| new TrieMap[SparkPlan, QueryStageExec]() | ||
| val stageCache: TrieMap[SparkPlan, ReusableQueryStageExec] = | ||
| new TrieMap[SparkPlan, ReusableQueryStageExec]() | ||
| } | ||
|
|
||
| /** | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
shall we mention that this can happen with table cache?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
sure, added