Skip to content

Conversation

@marin-ma
Copy link
Contributor

@marin-ma marin-ma commented Dec 4, 2025

Fix the transformer stage id contiguously increasing across different sql queries.

When AQE is off, the fix can be directly applied.

When AQE is on, vanillas spark set the rule CollapseCodegenStages to be stateful:
https://github.com/apache/spark/blob/branch-4.0/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala#L153-L156

However in Gluten, when AQE is on, the columnar rules are applied upon each individual query stages, and there's no stateful context shared across query stages that is visible to the columnar rules.

Seems like there's no way to use a stateful counter to generate incremental ids across different query stages. This pr adds a new rule to update the stage id after the physical plan is generated.

Related issue: #11251

@github-actions github-actions bot added CORE works for Gluten Core VELOX labels Dec 4, 2025
@github-actions
Copy link

github-actions bot commented Dec 4, 2025

Run Gluten Clickhouse CI on x86

@marin-ma marin-ma force-pushed the regenerate-transform-stageid branch from d3313ea to e45c64d Compare December 10, 2025 08:17
@github-actions
Copy link

Run Gluten Clickhouse CI on x86

1 similar comment
@github-actions
Copy link

Run Gluten Clickhouse CI on x86

@marin-ma marin-ma marked this pull request as ready for review December 10, 2025 14:30
@github-actions
Copy link

Run Gluten Clickhouse CI on x86

2 similar comments
@github-actions
Copy link

Run Gluten Clickhouse CI on x86

@github-actions
Copy link

Run Gluten Clickhouse CI on x86

@marin-ma
Copy link
Contributor Author

@zhztheplayer @zhouyuan Could you help to review? Thanks!

@marin-ma marin-ma changed the title [GLUTEN-11251] Fix incorrect whole stage id in WholeStageTransformerExec [GLUTEN-11251] Fix incorrect whole stage id in WholeStageTransformerExec Jan 5, 2026
@marin-ma marin-ma force-pushed the regenerate-transform-stageid branch from 75f416e to b23ff21 Compare January 5, 2026 16:37
@github-actions
Copy link

github-actions bot commented Jan 5, 2026

Run Gluten Clickhouse CI on x86


case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = false)(
val transformStageId: Int
var transformStageId: Int
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why changing to var?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When AQE is on, transformStageId always starts at 0 within each query stage. In this case transformStageId is not globally unique within the whole plan tree. After all query stages have been executed, the RegenerateTransformStageId rule traverses the whole plan tree and updates this value incrementally to a unique id.

* starts from 1. This rule updates the whole plan tree with incremental and unique transform stage
* id before the final execution.
*/
case class RegenerateTransformStageId() extends Rule[SparkPlan] with AdaptiveSparkPlanHelper {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Have you compared with vanilla Spark's query planner? I assume Spark doesn't need such rule for the correctness of the stage IDs in AQE?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Spark maintains a global counter in rule CollapseCodegenStages inside of the AdaptiveSparkPlanExec to make sure it generates unique id across query stages. https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala#L147-L150

But for Gluten we only can use a new counter for each query stage.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Understood. Though I feel the code of ID generation gets much harder to reason about after the PR's change.

Do you think it's feasible to change ColumnarCollapseTransformStages to leave all stage IDs unassigned (e.g., -1), then rely on RegenerateTransformStageId to assign all IDs, both for AQE and non-AQE cases? So it's clear that only RegenerateTransformStageId manages these IDs.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you think it's feasible to change ColumnarCollapseTransformStages to leave all stage IDs unassigned (e.g., -1), then rely on RegenerateTransformStageId to assign all IDs, both for AQE and non-AQE cases? So it's clear that only RegenerateTransformStageId manages these IDs.

Yes make sense.

case class ColumnarCollapseTransformStages(
glutenConf: GlutenConfig,
transformStageCounter: AtomicInteger = ColumnarCollapseTransformStages.transformStageCounter)
transformStageCounter: AtomicInteger = new AtomicInteger(0))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

private val transformStageCounter: AtomicInteger = new AtomicInteger(0)

private val wholeStageTransformerCache =
new mutable.HashSet[WholeStageTransformer]()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

HashSet[WholeStageTransformer]() is expensive. Can we use IdentityHashMap or so?

@github-actions
Copy link

github-actions bot commented Jan 6, 2026

Run Gluten Clickhouse CI on x86

1 similar comment
@github-actions
Copy link

github-actions bot commented Jan 6, 2026

Run Gluten Clickhouse CI on x86

@marin-ma marin-ma force-pushed the regenerate-transform-stageid branch from 4b56312 to 33ed32f Compare January 7, 2026 09:53
@github-actions
Copy link

github-actions bot commented Jan 7, 2026

Run Gluten Clickhouse CI on x86

@github-actions
Copy link

github-actions bot commented Jan 7, 2026

Run Gluten Clickhouse CI on x86

injector.injectPostTransform(
c => GlutenFallbackReporter(new GlutenConfig(c.sqlConf), c.session))
injector.injectPostTransform(_ => RemoveFallbackTagRule())
injector.injectPostTransform(_ => GenerateTransformStageId())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we move the rule right after ColumnarCollapseTransformStages? To make it clear that GenerateTransformStageId doesn't depend on other rules.

Copy link
Member

@zhztheplayer zhztheplayer left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks. Only a minor left from my end.

@github-actions
Copy link

github-actions bot commented Jan 7, 2026

Run Gluten Clickhouse CI on x86

@marin-ma marin-ma merged commit 60bf4b7 into apache:main Jan 7, 2026
110 of 113 checks passed
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 8, 2026
## Changes

| Cause | Type | Category | Description | Affected Files |
|-------|------|----------|-------------|----------------|
| N/A | Feat | Build | Update build configuration to support Spark 4.1 UT | `.github/workflows/velox_backend_x86.yml`, `gluten-ut/pom.xml`, `gluten-ut/spark41/pom.xml`, `tools/gluten-it/pom.xml` |
| [#52165](apache/spark#52165) | Fix | Dependency | Update Parquet dependency version to 1.16.0 to avoid NoSuchMethodError issue | `gluten-ut/spark41/pom.xml` |
| [#51477](apache/spark#51477) | Fix | Compatibility | Update imports to reflect streaming runtime package refactoring in Apache Spark | `gluten-ut/spark41/.../GlutenDynamicPartitionPruningSuite.scala`, `gluten-ut/spark41/.../GlutenStreamingQuerySuite.scala` |
| [#50674](apache/spark#50674) | Fix | Compatibility | Fix compatibility issue introduced by `TypedConfigBuilder` | `gluten-substrait/.../ExpressionConverter.scala`, `gluten-ut/spark41/.../GlutenCSVSuite.scala`, `gluten-ut/spark41/.../GlutenJsonSuite.scala` |
| [#49766](apache/spark#49766) | Fix | Compatibility | Disable V2 bucketing in GlutenDynamicPartitionPruningSuite since spark.sql.sources.v2.bucketing.enabled is now enabled by default | `gluten-ut/spark41/.../GlutenDynamicPartitionPruningSuite.scala` |
| [#42414](apache/spark#42414), [#53038](apache/spark#53038) | Fix | Bug Fix | Resolve an issue introduced by SPARK-42414, as identified in SPARK-53038 | `backends-velox/.../VeloxBloomFilterAggregate.scala` |
| N/A | Fix | Bug Fix | Enforce row fallback for unsupported cached batches - keep columnar execution only when schema validation succeeds | `backends-velox/.../ColumnarCachedBatchSerializer.scala` |
| [SPARK-53132](apache/spark#53132), [SPARK-53142](apache/spark#53142) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 KeyGroupedPartitioningSuite tests. Excluded tests: `SPARK-53322*`, `SPARK-54439*` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [SPARK-53535](https://issues.apache.org/jira/browse/SPARK-53535), [SPARK-54220](https://issues.apache.org/jira/browse/SPARK-54220) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenParquetIOSuite tests. Excluded tests: `SPARK-53535*`, `vectorized reader: missing all struct fields*`, `SPARK-54220*` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#52645](apache/spark#52645) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenStreamingQuerySuite tests. Excluded tests: `SPARK-53942: changing the number of stateless shuffle partitions via config`, `SPARK-53942: stateful shuffle partitions are retained from old checkpoint` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#47856](apache/spark#47856) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenDataFrameWindowFunctionsSuite and GlutenJoinSuite tests. Excluded tests: `SPARK-49386: Window spill with more than the inMemoryThreshold and spillSizeThreshold`, `SPARK-49386: test SortMergeJoin (with spill by size threshold)` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#52157](apache/spark#52157) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenQueryExecutionSuite tests. Excluded test: `#53413: Cleanup shuffle dependencies for commands` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#48470](apache/spark#48470) | 4.1.0 | Test Exclusion | Exclude split test in GlutenRegexpExpressionsSuite. Excluded test: `GlutenRegexpExpressionsSuite.SPLIT` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#51623](apache/spark#51623) | 4.1.0 | Test Exclusion | Add `spark.sql.unionOutputPartitioning=false` to Maven test args. Excluded tests: `GlutenBroadcastExchangeSuite.SPARK-52962`, `GlutenDataFrameSetOperationsSuite.SPARK-52921*` | `.github/workflows/velox_backend_x86.yml`, `gluten-ut/spark41/.../VeloxTestSettings.scala`, `tools/gluten-it/common/.../Suite.scala` |
| N/A | 4.1.0 | Test Exclusion | Excludes failed SQL tests that need to be fixed for Spark 4.1 compatibility. Excluded tests: `decimalArithmeticOperations.sql`, `identifier-clause.sql`, `keywords.sql`, `literals.sql`, `operators.sql`, `exists-orderby-limit.sql`, `postgreSQL/date.sql`, `nonansi/keywords.sql`, `nonansi/literals.sql`, `datetime-legacy.sql`, `datetime-parsing-invalid.sql`, `misc-functions.sql` | `gluten-ut/spark41/.../VeloxSQLQueryTestSettings.scala` |
| apache#11252 | 4.1.0 | Test Exclusion | Exclude Gluten test for SPARK-47939: Explain should work with parameterized queries |  `gluten-ut/spark41/.../VeloxTestSettings.scala` |
QCLyu pushed a commit to QCLyu/incubator-gluten that referenced this pull request Jan 8, 2026
QCLyu pushed a commit to QCLyu/incubator-gluten that referenced this pull request Jan 8, 2026
baibaichen added a commit that referenced this pull request Jan 8, 2026
## Changes

| Cause | Type | Category | Description | Affected Files |
|-------|------|----------|-------------|----------------|
| N/A | Feat | Build | Update build configuration to support Spark 4.1 UT | `.github/workflows/velox_backend_x86.yml`, `gluten-ut/pom.xml`, `gluten-ut/spark41/pom.xml`, `tools/gluten-it/pom.xml` |
| [#52165](apache/spark#52165) | Fix | Dependency | Update Parquet dependency version to 1.16.0 to avoid NoSuchMethodError issue | `gluten-ut/spark41/pom.xml` |
| [#51477](apache/spark#51477) | Fix | Compatibility | Update imports to reflect streaming runtime package refactoring in Apache Spark | `gluten-ut/spark41/.../GlutenDynamicPartitionPruningSuite.scala`, `gluten-ut/spark41/.../GlutenStreamingQuerySuite.scala` |
| [#50674](apache/spark#50674) | Fix | Compatibility | Fix compatibility issue introduced by `TypedConfigBuilder` | `gluten-substrait/.../ExpressionConverter.scala`, `gluten-ut/spark41/.../GlutenCSVSuite.scala`, `gluten-ut/spark41/.../GlutenJsonSuite.scala` |
| [#49766](apache/spark#49766) | Fix | Compatibility | Disable V2 bucketing in GlutenDynamicPartitionPruningSuite since spark.sql.sources.v2.bucketing.enabled is now enabled by default | `gluten-ut/spark41/.../GlutenDynamicPartitionPruningSuite.scala` |
| [#42414](apache/spark#42414), [#53038](apache/spark#53038) | Fix | Bug Fix | Resolve an issue introduced by SPARK-42414, as identified in SPARK-53038 | `backends-velox/.../VeloxBloomFilterAggregate.scala` |
| N/A | Fix | Bug Fix | Enforce row fallback for unsupported cached batches - keep columnar execution only when schema validation succeeds | `backends-velox/.../ColumnarCachedBatchSerializer.scala` |
| [SPARK-53132](apache/spark#53132), [SPARK-53142](apache/spark#53142) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 KeyGroupedPartitioningSuite tests. Excluded tests: `SPARK-53322*`, `SPARK-54439*` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [SPARK-53535](https://issues.apache.org/jira/browse/SPARK-53535), [SPARK-54220](https://issues.apache.org/jira/browse/SPARK-54220) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenParquetIOSuite tests. Excluded tests: `SPARK-53535*`, `vectorized reader: missing all struct fields*`, `SPARK-54220*` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#52645](apache/spark#52645) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenStreamingQuerySuite tests. Excluded tests: `SPARK-53942: changing the number of stateless shuffle partitions via config`, `SPARK-53942: stateful shuffle partitions are retained from old checkpoint` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#47856](apache/spark#47856) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenDataFrameWindowFunctionsSuite and GlutenJoinSuite tests. Excluded tests: `SPARK-49386: Window spill with more than the inMemoryThreshold and spillSizeThreshold`, `SPARK-49386: test SortMergeJoin (with spill by size threshold)` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#52157](apache/spark#52157) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenQueryExecutionSuite tests. Excluded test: `#53413: Cleanup shuffle dependencies for commands` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#48470](apache/spark#48470) | 4.1.0 | Test Exclusion | Exclude split test in GlutenRegexpExpressionsSuite. Excluded test: `GlutenRegexpExpressionsSuite.SPLIT` | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
| [#51623](apache/spark#51623) | 4.1.0 | Test Exclusion | Add `spark.sql.unionOutputPartitioning=false` to Maven test args. Excluded tests: `GlutenBroadcastExchangeSuite.SPARK-52962`, `GlutenDataFrameSetOperationsSuite.SPARK-52921*` | `.github/workflows/velox_backend_x86.yml`, `gluten-ut/spark41/.../VeloxTestSettings.scala`, `tools/gluten-it/common/.../Suite.scala` |
| N/A | 4.1.0 | Test Exclusion | Excludes failed SQL tests that need to be fixed for Spark 4.1 compatibility. Excluded tests: `decimalArithmeticOperations.sql`, `identifier-clause.sql`, `keywords.sql`, `literals.sql`, `operators.sql`, `exists-orderby-limit.sql`, `postgreSQL/date.sql`, `nonansi/keywords.sql`, `nonansi/literals.sql`, `datetime-legacy.sql`, `datetime-parsing-invalid.sql`, `misc-functions.sql` | `gluten-ut/spark41/.../VeloxSQLQueryTestSettings.scala` |
| #11252 | 4.1.0 | Test Exclusion | Exclude Gluten test for SPARK-47939: Explain should work with parameterized queries |  `gluten-ut/spark41/.../VeloxTestSettings.scala` |
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

CLICKHOUSE CORE works for Gluten Core VELOX

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants