Skip to content

Conversation

@dongjoon-hyun
Copy link
Member

@dongjoon-hyun dongjoon-hyun commented Feb 3, 2025

What changes were proposed in this pull request?

This PR aims to enable spark.sql.sources.v2.bucketing.enabled by default for Apache Spark 4.1.0.

Why are the changes needed?

We have been using spark.sql.sources.v2.bucketing.enabled since Apache Spark 3.3.0 stably to improve Spark performance on V2 data source. Although Apache Spark enables this configuration, Spark checks if all InputPartitions have HasPartitionKey or not still.

if (results.length != inputPartitions.length || inputPartitions.isEmpty) {
// Not all of the `InputPartitions` implements `HasPartitionKey`, therefore skip here.
None

Does this PR introduce any user-facing change?

No.

How was this patch tested?

Pass the CIs.

Was this patch authored or co-authored using generative AI tooling?

No.

@github-actions github-actions bot added the SQL label Feb 3, 2025
@dongjoon-hyun dongjoon-hyun changed the title [SPARK-51064][SQL] Enable spark.sql.sources.v2.bucketing.enabled by… [SPARK-51064][SQL] Enable spark.sql.sources.v2.bucketing.enabled by default Feb 3, 2025
@dongjoon-hyun
Copy link
Member Author

What do you think about this change, @sunchao and @szehon-ho ?

@dongjoon-hyun dongjoon-hyun marked this pull request as ready for review February 3, 2025 18:32
@github-actions github-actions bot added the DOCS label Feb 3, 2025
@dongjoon-hyun
Copy link
Member Author

Also, cc @aokolnychyi , @viirya , @huaxingao . I'd like to get your opinions on this.

Copy link
Member

@sunchao sunchao left a comment

Choose a reason for hiding this comment

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

thanks @dongjoon-hyun ! +1 on making this as the default config - the feature has been stabilized for several release cycles already.

@viirya
Copy link
Member

viirya commented Feb 4, 2025

As it is used since Apache Spark 3.3.0, the feature should be stable enough for now. Except for any concerns from others, I'm okay for this change.

@dongjoon-hyun
Copy link
Member Author

Thank you, @sunchao and @viirya .
Merged to master for Apache Spark 4.1.0.

@dongjoon-hyun dongjoon-hyun deleted the SPARK-51064 branch February 4, 2025 17:36
@guangyu-yang-rokt
Copy link

Hi @dongjoon-hyun, sorry this might be a question a bit unrelated to this PR.

Context:
We are currently introducing SPJ to our production environment. Our iceberg table is partitioned by timestamp with day transformation and in our ML processing job we will read past 30 days worth of data with filter on timestamp column which will be pushed down to iceberg. So 30 partitions will be reported by iceberg. I have observed that with spark.sql.sources.v2.bucketing.enabled, spark will then generate one task per partition, which will be only 30 tasks in our case when we are doing batchScan. This has led to cluster resource under utilisation since we have 40 execs and 15 cores each (so at max 600 tasks in parallel). That impact the batchScan performance a lot - same stage from 2.4 mins to 10+mins.

Have you encountered the same issue in your use case? I must be missing something here. Any insights will be much appreciated!

@szehon-ho
Copy link
Member

@guangyu-yang-rokt yea its right, by design in SPJ the number of Iceberg partitions is equal to number of Spark partitions, so sizing that is critical. There is an option to enable more parallelism in spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled, but it leads to more times the data read, and doesnt work in all cases (not FULL OUTER joins)

@guangyu-yang-rokt
Copy link

Thanks @szehon-ho! one follow up question - in our query, we only do filtering on timestamp column but join key is something different (joining on non-partition keys). I have checked that BatchScanExec is reporting groupedBy=[timestamp_day] in query plan. I'm not too familiar with spark codebase but I guess filter pushdown to iceberg also tell BatchScanExec to group by partition key if there is a filter on partition key. With spark.sql.sources.v2.bucketing.enabledset to true, it will slow down batchScan for joins that are not joining on partition keys. (we have a self-implemented featurestore which will spin up multiple joins to gather features in so I need to enable all SPJ related configs globally)

This is kind not making sense to me since I'm not joining on timestamp so I would expect SPJ shouldn't kick in. Or I would imagine a configuration like spark.sql.sources.v2.ignoreFiltering to tell BatchScanExec don't grouped by partition key if it is just a filter and not a join key

baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 22, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 22, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 22, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 30, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 5, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 5, 2026
apache/spark#49766, since spark.sql.sources.v2.bucketing.enabled is now enabled by default.
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 7, 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` |
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 7, 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` |
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 7, 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` |
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` |
baibaichen added a commit to apache/incubator-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` |
| #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

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants