sezruby opened a new pull request, #12240:
URL: https://github.com/apache/gluten/pull/12240

   ## 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)
   
   ```scala
   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
   
   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 in the file. 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 compares names.
   
   | Path | Pre-fix | Post-fix |
   |---|---|---|
   | Delta partition pruning (`PreparedDeltaFileIndex.matchingFiles`) | broken 
— physical-named filter, logical-named schema | works — logical-named on both 
sides |
   | Delta file-level stats skipping (`Snapshot.filesForScan`) | broken | works 
|
   | Velox parquet read (`output` / `dataSchema`) | works (physical) | works 
(physical) |
   | Velox filter pushdown (`scanFilters` / `pushDownFilters`) | works (both 
physical) | works — `scanFilters` translates logical→physical for the native 
side |
   
   The fix is Gluten-only — no Delta-side changes.
   
   ## 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. Also 
asserts only one partition is selected after pruning (regression for the bug 
report).
   - **`column mapping mode = $mode with partition filter (multi partition 
col)`** — predicate spanning two partition columns.
   - **`column mapping mode = $mode with partition + data filter`** — partition 
pruning + data column predicate together; also a data-only predicate to 
exercise file-level stats skipping.
   - **`column mapping mode = $mode with IS [NOT] NULL on partition col`** — 
null-partition handling.
   - **`column mapping mode = $mode partition filter survives column rename`** 
— `ALTER TABLE ... RENAME COLUMN` on the partition column; logical name 
changes, physical stays the same.
   - **`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.
   
   The existing tests (`column mapping mode = id`, `column mapping mode = 
name`, `column mapping with complex type`) continue to pass.
   
   ## Was this patch authored or co-authored using generative AI tooling?
   
   Generated-by: Claude Code (Claude Opus 4.7)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to