-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-32430][SQL] Extend SparkSessionExtensions to inject rules into AQE query stage preparation #29224
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
[SPARK-32430][SQL] Extend SparkSessionExtensions to inject rules into AQE query stage preparation #29224
Changes from all commits
d08003c
3a8309c
dcfabc5
630e9f8
e884516
68709ea
182d4f9
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 |
|---|---|---|
|
|
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.ExpressionInfo | |
| import org.apache.spark.sql.catalyst.parser.ParserInterface | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.execution.ColumnarRule | ||
| import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
|
|
@@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.ColumnarRule | |
| * <li>Customized Parser.</li> | ||
| * <li>(External) Catalog listeners.</li> | ||
| * <li>Columnar Rules.</li> | ||
| * <li>Adaptive Query Stage Preparation Rules.</li> | ||
| * </ul> | ||
| * | ||
| * The extensions can be used by calling `withExtensions` on the [[SparkSession.Builder]], for | ||
|
|
@@ -96,8 +97,10 @@ class SparkSessionExtensions { | |
| type ParserBuilder = (SparkSession, ParserInterface) => ParserInterface | ||
| type FunctionDescription = (FunctionIdentifier, ExpressionInfo, FunctionBuilder) | ||
| type ColumnarRuleBuilder = SparkSession => ColumnarRule | ||
| type QueryStagePrepRuleBuilder = SparkSession => Rule[SparkPlan] | ||
|
Member
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. This is a public API. I think we also need to add a version information.
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. It is marked as unstable and experimental. Do we need to add version information in that case? I am fine with doing so, but then we should add it everywhere. |
||
|
|
||
| private[this] val columnarRuleBuilders = mutable.Buffer.empty[ColumnarRuleBuilder] | ||
| private[this] val queryStagePrepRuleBuilders = mutable.Buffer.empty[QueryStagePrepRuleBuilder] | ||
|
|
||
| /** | ||
| * Build the override rules for columnar execution. | ||
|
|
@@ -106,13 +109,28 @@ class SparkSessionExtensions { | |
| columnarRuleBuilders.map(_.apply(session)).toSeq | ||
| } | ||
|
|
||
| /** | ||
| * Build the override rules for the query stage preparation phase of adaptive query execution. | ||
| */ | ||
| private[sql] def buildQueryStagePrepRules(session: SparkSession): Seq[Rule[SparkPlan]] = { | ||
| queryStagePrepRuleBuilders.map(_.apply(session)).toSeq | ||
| } | ||
|
|
||
| /** | ||
| * Inject a rule that can override the columnar execution of an executor. | ||
| */ | ||
| def injectColumnar(builder: ColumnarRuleBuilder): Unit = { | ||
| columnarRuleBuilders += builder | ||
| } | ||
|
|
||
| /** | ||
| * Inject a rule that can override the the query stage preparation phase of adaptive query | ||
|
Member
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. Nit: the the => the |
||
| * execution. | ||
| */ | ||
| def injectQueryStagePrepRule(builder: QueryStagePrepRuleBuilder): Unit = { | ||
| queryStagePrepRuleBuilders += builder | ||
| } | ||
|
|
||
| private[this] val resolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder] | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -90,7 +90,7 @@ case class AdaptiveSparkPlanExec( | |
| // Exchange nodes) after running these rules. | ||
| private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( | ||
|
Member
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. It sounds like this function should be moved out of the physical node |
||
| ensureRequirements | ||
| ) | ||
| ) ++ context.session.sessionState.queryStagePrepRules | ||
|
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. Should we do this after ensure requirements? The
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. The custom rules may need to see exchange nodes as well. We can do validation at the end to make sure exchange requirements are still required. |
||
|
|
||
| // A list of physical optimizer rules to be applied to a new stage before its execution. These | ||
| // optimizations should be stage-independent. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.catalog._ | |
| import org.apache.spark.sql.catalyst.optimizer.Optimizer | ||
| import org.apache.spark.sql.catalyst.parser.ParserInterface | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.connector.catalog.CatalogManager | ||
| import org.apache.spark.sql.execution._ | ||
| import org.apache.spark.sql.streaming.StreamingQueryManager | ||
|
|
@@ -73,7 +74,8 @@ private[sql] class SessionState( | |
| resourceLoaderBuilder: () => SessionResourceLoader, | ||
| createQueryExecution: LogicalPlan => QueryExecution, | ||
| createClone: (SparkSession, SessionState) => SessionState, | ||
| val columnarRules: Seq[ColumnarRule]) { | ||
| val columnarRules: Seq[ColumnarRule], | ||
| val queryStagePrepRules: Seq[Rule[SparkPlan]]) { | ||
|
Member
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. SessionState is a critical concept/class in Spark SQL. Adding cc @hvanhovell
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. Preferably both
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. It's very hard. The extension rules are injected in the
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. It is just refactoring right (famous last words)? You pull out the rules from query execution, put them in the session state builder, and manipulate them there.
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. Maybe I missed something. The session state builder is used to build Are you suggesting we create something like |
||
|
|
||
| // The following fields are lazy to avoid creating the Hive client when creating SessionState. | ||
| lazy val catalog: SessionCatalog = catalogBuilder() | ||
|
|
||
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.
Please update the document (at line 47) accordingly.
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.
Thanks. I pushed an update for this.