Skip to content

Conversation

@peter-toth
Copy link
Contributor

What changes were proposed in this pull request?

This PR fixes the case when a scalar subquery with a BloomFilterMightContain expression is added to PlanMerger, but its bloomFilterExpression contains a merged scalar subquery which was replaced to a ScalarSubqueryReference before.
In this case BloomFilterMightContain was unresolved due to its checkInputDataTypes() failure.

Why are the changes needed?

To

Does this PR introduce any user-facing change?

No.

How was this patch tested?

New UT.

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

No.

@github-actions github-actions bot added the SQL label Nov 13, 2025
@peter-toth peter-toth changed the title [SPARK-54336][SQL] Fix BloomFilterMightContain type check with ScalarSubqueryReference [SPARK-54336][SQL] Fix BloomFilterMightContain input type check with ScalarSubqueryReference Nov 13, 2025
Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

+1, LGTM. (Pending CIs). Thank you, @peter-toth .

@dongjoon-hyun
Copy link
Member

According to the Affected Versions of JIRA, we want to merge this to master branch only?

Screenshot 2025-11-13 at 08 36 25

@peter-toth
Copy link
Contributor Author

I'm still checking which branches are affected, will update the ticket...

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-54336][SQL] Fix BloomFilterMightContain input type check with ScalarSubqueryReference [SPARK-54336][SQL] Fix BloomFilterMightContain input type check with ScalarSubqueryReference Nov 13, 2025
@peter-toth
Copy link
Contributor Author

@dongjoon-hyun, we need this fix on all active branches. I've updated the ticket.

@dongjoon-hyun
Copy link
Member

Got it. Thank you for checking it, @peter-toth .

dongjoon-hyun pushed a commit that referenced this pull request Nov 13, 2025
…h `ScalarSubqueryReference`

### What changes were proposed in this pull request?

This PR fixes the case when a scalar subquery with a `BloomFilterMightContain` expression is added to `PlanMerger`, but its `bloomFilterExpression` contains a merged scalar subquery which was replaced to a `ScalarSubqueryReference` before.
In this case `BloomFilterMightContain` was unresolved due to its `checkInputDataTypes()` failure.

### Why are the changes needed?
To

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT.

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

No.

Closes #53038 from peter-toth/SPARK-54336-fix-bloomfiltermightcontain-type-check.

Authored-by: Peter Toth <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
(cherry picked from commit 06b3d4c)
Signed-off-by: Dongjoon Hyun <[email protected]>
dongjoon-hyun pushed a commit that referenced this pull request Nov 13, 2025
…h `ScalarSubqueryReference`

### What changes were proposed in this pull request?

This PR fixes the case when a scalar subquery with a `BloomFilterMightContain` expression is added to `PlanMerger`, but its `bloomFilterExpression` contains a merged scalar subquery which was replaced to a `ScalarSubqueryReference` before.
In this case `BloomFilterMightContain` was unresolved due to its `checkInputDataTypes()` failure.

### Why are the changes needed?
To

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT.

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

No.

Closes #53038 from peter-toth/SPARK-54336-fix-bloomfiltermightcontain-type-check.

Authored-by: Peter Toth <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
(cherry picked from commit 06b3d4c)
Signed-off-by: Dongjoon Hyun <[email protected]>
dongjoon-hyun pushed a commit that referenced this pull request Nov 13, 2025
…h `ScalarSubqueryReference`

### What changes were proposed in this pull request?

This PR fixes the case when a scalar subquery with a `BloomFilterMightContain` expression is added to `PlanMerger`, but its `bloomFilterExpression` contains a merged scalar subquery which was replaced to a `ScalarSubqueryReference` before.
In this case `BloomFilterMightContain` was unresolved due to its `checkInputDataTypes()` failure.

### Why are the changes needed?
To

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT.

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

No.

Closes #53038 from peter-toth/SPARK-54336-fix-bloomfiltermightcontain-type-check.

Authored-by: Peter Toth <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
(cherry picked from commit 06b3d4c)
Signed-off-by: Dongjoon Hyun <[email protected]>
@dongjoon-hyun
Copy link
Member

After I manually ran the newly added test case on all release branches, I merged to master/4.1/4.0/3.5.

@peter-toth
Copy link
Contributor Author

After I manually ran the newly added test case on all release branches, I merged to master/4.1/4.0/3.5.

Thank you @dongjoon-hyun!

zifeif2 pushed a commit to zifeif2/spark that referenced this pull request Nov 22, 2025
…h `ScalarSubqueryReference`

### What changes were proposed in this pull request?

This PR fixes the case when a scalar subquery with a `BloomFilterMightContain` expression is added to `PlanMerger`, but its `bloomFilterExpression` contains a merged scalar subquery which was replaced to a `ScalarSubqueryReference` before.
In this case `BloomFilterMightContain` was unresolved due to its `checkInputDataTypes()` failure.

### Why are the changes needed?
To

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT.

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

No.

Closes apache#53038 from peter-toth/SPARK-54336-fix-bloomfiltermightcontain-type-check.

Authored-by: Peter Toth <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
huangxiaopingRD pushed a commit to huangxiaopingRD/spark that referenced this pull request Nov 25, 2025
…h `ScalarSubqueryReference`

### What changes were proposed in this pull request?

This PR fixes the case when a scalar subquery with a `BloomFilterMightContain` expression is added to `PlanMerger`, but its `bloomFilterExpression` contains a merged scalar subquery which was replaced to a `ScalarSubqueryReference` before.
In this case `BloomFilterMightContain` was unresolved due to its `checkInputDataTypes()` failure.

### Why are the changes needed?
To

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT.

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

No.

Closes apache#53038 from peter-toth/SPARK-54336-fix-bloomfiltermightcontain-type-check.

Authored-by: Peter Toth <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 22, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 22, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 22, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 23, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Dec 31, 2025
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 4, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 5, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 5, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 5, 2026
baibaichen added a commit to baibaichen/gluten that referenced this pull request Jan 5, 2026
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

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants