Skip to content

[GLUTEN-10511][VL][Delta] Fix wrong result with partition filters under column mapping#12240

Merged
zhztheplayer merged 7 commits into
apache:mainfrom
sezruby:fix-delta-cm-partition-filter
Jun 9, 2026
Merged

[GLUTEN-10511][VL][Delta] Fix wrong result with partition filters under column mapping#12240
zhztheplayer merged 7 commits into
apache:mainfrom
sezruby:fix-delta-cm-partition-filter

Conversation

@sezruby

@sezruby sezruby commented Jun 4, 2026

Copy link
Copy Markdown
Contributor

What changes are proposed in this pull request?

Fixes #10511.

When delta.columnMapping.mode is name or id, a query with a partition column filter could return all rows instead of the pruned set. The same mechanism also disabled file-level data skipping under column mapping (a silent perf regression that masked the same root cause).

Reproducer (from the issue)

spark.sql("""
  create table delta_cm2 (id int, name string) using delta
  partitioned by (id)
  tblproperties ("delta.columnMapping.mode" = "name")""")
spark.sql("insert into delta_cm2 values (1, 'v1'), (2, 'v2'), (3, 'v3')")
spark.sql("select name from delta_cm2 where id > 2").show()
// Expected: ["v3"]
// Actual (Gluten/Velox): ["v1", "v2", "v3"]

Root cause

DeltaPostTransformRules.transformColumnMappingPlan rewrote partitionFilters, dataFilters, partitionSchema, and requiredSchema from logical → physical column names so the parquet reader sees physical names. But Delta's PreparedDeltaFileIndex.matchingFiles / Snapshot.filesForScan and DeltaLog.rewritePartitionFilters resolve filter attributes against the logical metadata.partitionSchema and the logical column-stats schema. Once filter attributes were physical-named:

  • rewritePartitionFilters could not match the physical attribute against the logical partition schema, fell into the case None branch, and emitted a bare UnresolvedAttribute without a Cast — partition pruning silently no-op'd, all files were returned.
  • File-level stats skipping silently missed all files for the same reason on the data side.

Vanilla Spark + Delta resolves this asymmetry inside DeltaParquetFileFormat.buildReaderWithPartitionValues (which translates filters/schema to physical only when handing to the parquet reader). Gluten bypasses that hook, so the rule rewrote everything uniformly — too aggressively.

Fix

The scan node now has a deliberate logical/physical split:

Field After fix Why
output, dataSchema, data fields of requiredSchema physical Drives the substrait NamedStruct Velox uses to find columns in parquet.
partitionSchema, partitionFilters, dataFilters, partition fields of requiredSchema logical Consumed by Delta's PreparedDeltaFileIndex / Snapshot.filesForScan for partition pruning + stats-based file skipping. Delta resolves these against logical schemas.
DeltaScanTransformer.scanFilters (override) physical, translated from dataFilters by exprId match against output BasicScanExecTransformer.filterExprs() matches scanFilters against the physical-named pushDownFilters from the upstream Filter by AttributeReference.equals (which compares names).

The fix is Gluten-only — no Delta-side changes.

Why the design looks asymmetric

Vanilla Spark + Delta keeps everything on the scan node logical and translates physical-only inside DeltaParquetFileFormat.buildReaderWithPartitionValues. Gluten bypasses that hook (it goes to native via Substrait), so the translation has to live somewhere on our side. The rule today does this translation eagerly on the scan node for the reader-facing fields, which is the smallest delta from the pre-fix shape and explains why some fields are physical and others logical.

A cleaner shape — possible follow-up — is to keep ALL scan-node fields logical and translate only at substrait emission time (inside the NamedStruct/ReadRel build in BasicScanExecTransformer.doTransform). That removes both the alias-back ProjectExecTransformer and the scanFilters override, but it requires plumbing Delta-specific physical-name lookup into the substrait emitter. Out of scope for this bug fix; noted in the docstring of transformColumnMappingPlan.

How was this patch tested?

DeltaSuite is extended with new tests, all parameterized over both name and id column-mapping modes:

  • column mapping mode = $mode with partition filter (single partition col) — equality, range, and IN predicates on a partition column. Asserts df.inputFiles.length (Delta-level pruning) and DeltaScanTransformer.getPartitionArray.length (post-Gluten-rewrite selected partition count) both reflect the pruned set. Includes the exact case from the bug report.
  • column mapping mode = $mode with partition filter (multi partition col) — predicate spanning two partition columns; predicate on only one of two.
  • column mapping mode = $mode with partition + data filter — partition pruning + data column predicate together; data-only predicate exercises file-level stats skipping (asserts the file count actually shrunk).
  • column mapping mode = $mode with IS [NOT] NULL on partition col — null-partition handling.
  • column mapping mode = $mode partition filter survives column renameALTER TABLE ... RENAME COLUMN on the partition column.
  • column mapping mode = $mode data column rename + filter (file skipping)ALTER TABLE ... RENAME COLUMN on a data column; ensures filter pushdown still resolves to the physical column in parquet AND Delta's stats-based skipping resolves the new logical name.

The existing tests (column mapping mode = id, column mapping mode = name, column mapping with complex type) continue to pass.

Verified locally end-to-end: pulled apache/gluten:centos-8-jdk8, downloaded the prebuilt CI artifacts (velox-native-lib, arrow-jars), ran mvn test -pl backends-velox -DwildcardSuites=org.apache.gluten.execution.VeloxDeltaSuite. 30/30 tests pass, 0 failures, 0 errors, including all 12 new tests.

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

Generated-by: Claude Code (Claude Opus 4.7)

…er column mapping

When `delta.columnMapping.mode` is `name` or `id`, a query with a partition
column filter could return all rows instead of the pruned set. The same
mechanism also disabled file-level data skipping.

Root cause is in `DeltaPostTransformRules.transformColumnMappingPlan`. The
rule rewrote `partitionFilters`, `dataFilters`, `partitionSchema`, and
`requiredSchema` from logical to physical column names so the parquet reader
sees physical names. But Delta's `PreparedDeltaFileIndex.matchingFiles` /
`Snapshot.filesForScan` and `DeltaLog.rewritePartitionFilters` resolve filter
attributes against the *logical* `metadata.partitionSchema` and the logical
column-stats schema. Once attributes were physical-named:

- `rewritePartitionFilters` could not match the physical attribute against the
  logical partition schema, fell into the `case None` branch, and emitted a
  bare `UnresolvedAttribute` without a `Cast` — partition pruning silently
  no-op'd, all files were returned.
- File-level stats skipping silently missed all files (perf regression that
  masked the same root cause).

Fix: keep filter attrs and partition schema LOGICAL on the scan node so
Delta's file index resolves them correctly. Reader-facing pieces (`output`,
`dataSchema`, the data fields of `requiredSchema`) stay PHYSICAL so the
parquet reader and Velox find the right columns. Filter binding to the
native side is by exprId, not by name, so logical-named filter attrs still
resolve correctly against the physical-named `output`.

`DeltaScanTransformer.scanFilters` is overridden to translate the logical
`dataFilters` to their physical-named counterparts (by exprId match against
`output`) for the native side, since `BasicScanExecTransformer.filterExprs()`
matches `scanFilters` against `pushDownFilters` (built over the physical-named
scan output) by `AttributeReference.equals`, which checks names.

Tests added in `DeltaSuite` cover both `name` and `id` modes for: equality /
range / IN partition filters, multi-partition-column filters, combined
partition+data filters, IS [NOT] NULL on partition columns, and column-rename
scenarios for both partition and data columns.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@github-actions

github-actions Bot commented Jun 4, 2026

Copy link
Copy Markdown

Run Gluten Clickhouse CI on x86

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@github-actions

github-actions Bot commented Jun 4, 2026

Copy link
Copy Markdown

Run Gluten Clickhouse CI on x86

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@github-actions

github-actions Bot commented Jun 4, 2026

Copy link
Copy Markdown

Run Gluten Clickhouse CI on x86

…counts

Assert both `df.inputFiles.length` (Delta-level pruning) and
`DeltaScanTransformer.getPartitions.size` (post-Gluten-rewrite split count) on
every test, so a regression that disables either layer is caught.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@github-actions

github-actions Bot commented Jun 4, 2026

Copy link
Copy Markdown

Run Gluten Clickhouse CI on x86

…ssertion

`getPartitions` reflects post-coalesce splits (Velox merges small per-partition
files into one input split), which made the assertion flaky for tables with
tiny files. Switch to `getPartitionArray` -- the pre-coalesce list of partition
directories selected by the executed scan -- which is the actual call site
exercised by the bug fix.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@github-actions

github-actions Bot commented Jun 4, 2026

Copy link
Copy Markdown

Run Gluten Clickhouse CI on x86

…eanup

Expand the docstring of `transformColumnMappingPlan` to explain why some scan
node fields stay logical and others become physical, and note the longer-term
cleanup direction (defer all physical translation to substrait emission, which
would remove both the alias-back project and the scanFilters override).

Mirror the same context in the `DeltaScanTransformer.scanFilters` override so
each site reads independently.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@github-actions

github-actions Bot commented Jun 5, 2026

Copy link
Copy Markdown

Run Gluten Clickhouse CI on x86

@sezruby

sezruby commented Jun 5, 2026

Copy link
Copy Markdown
Contributor Author

@zhztheplayer @FelixYBW could you take a look when you get a chance?

Quick context: this fixes the wrong-result bug in #10511 by narrowing the column-mapping rewrite — only the reader-facing fields (output, dataSchema, data part of requiredSchema) become physical; partition schema and filters stay logical so Delta's PreparedDeltaFileIndex keeps working for partition pruning + stats-based file skipping. Native side gets a physical-translated filter copy via a scanFilters override.

The result is asymmetric on the scan node, which I know is a bit ugly. The reason is that vanilla Spark + Delta does the logical→physical translation just-in-time inside DeltaParquetFileFormat.buildReaderWithPartitionValues, and Gluten bypasses that hook. The cleaner shape would be to keep EVERYTHING on the scan node logical and translate only at substrait emission (BasicScanExecTransformer.doTransform) — that would also let us drop the alias-back ProjectExecTransformer and the scanFilters override. But that's a multi-module refactor (touches the substrait emitter shared across Iceberg/Hudi/plain Parquet/Delta), so I left it as a follow-up noted in the docstring rather than scope-creep into a bug fix.

Verified locally end-to-end with the prebuilt CI artifacts in apache/gluten:centos-8-jdk8VeloxDeltaSuite passes 30/30 including all 12 new tests. CI is also green on the latest commit.

@sezruby

sezruby commented Jun 8, 2026

Copy link
Copy Markdown
Contributor Author

@zhztheplayer @FelixYBW can you prioritize the PR as it's a silent wrong result issue? Thanks

@github-actions

github-actions Bot commented Jun 9, 2026

Copy link
Copy Markdown

Run Gluten Clickhouse CI on x86

@zhztheplayer zhztheplayer merged commit bdaad2a into apache:main Jun 9, 2026
61 of 62 checks passed
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.

Delta column mapping mode caused wrong result with partition column filters

3 participants