Skip to content

Fix RowNumberReader when not all row groups are selected#8863

Merged
alamb merged 6 commits into
apache:mainfrom
vustef:vs-fix-row-nums
Nov 18, 2025
Merged

Fix RowNumberReader when not all row groups are selected#8863
alamb merged 6 commits into
apache:mainfrom
vustef:vs-fix-row-nums

Conversation

@vustef

@vustef vustef commented Nov 18, 2025

Copy link
Copy Markdown
Contributor

Which issue does this PR close?

Closes #8864.

Rationale for this change

#8715 introduced row numbers feature last week. However, it had a bug, which luckily for us @scovich pointed out soon after the merge. The issue is that the row numbers are produced in ordinal-based order of the row groups, instead of user-requested order of row groups. The former is wrong, and is being fixed here by switching to user-requested order.

What changes are included in this PR?

Just fixing the bug as explained above, and adding test. Also addressing two small comments from post-merge review: #8715 (review)

Are these changes tested?

Yes.

Are there any user-facing changes?

No, this wasn't released yet.

@github-actions github-actions Bot added the parquet Changes to the parquet crate label Nov 18, 2025
@vustef

vustef commented Nov 18, 2025

Copy link
Copy Markdown
Contributor Author

@scovich @alamb this is a bug fix on top of previous PR. I hope we can review it in time for 57.1.0 release candidate that was supposed to be carved out yesterday.

@@ -625,12 +625,11 @@ impl ArrowReaderOptions {
pub fn with_virtual_columns(self, virtual_columns: Vec<FieldRef>) -> Self {

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I also made this comment in the original PR, but maybe we could change this to

pub fn set_virtual_columns(&mut self, virtual_columns: Vec<FieldRef>) -> Result<()> {

to avoid the panic. If we squeak this in before 57.1.0 ships it wouldn't be an API change.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Attaching a link for easier discussion: #8715 (comment)

Not having an ability to chain seems a bit degrading, though users can just use ? operator for that (in which case it just panics, but that's fine, it's a choice).
Having a proper builder would be nice, but it's then definitely an API change, and a bit less quick fix for the main thing that this PR is intended to solve.

I'm fine with changing to Result(). Would you prefer that? Are others on board?

@scovich scovich Nov 18, 2025

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why is Result<Self> bad, sorry? Call it try_with_xxx if naming conventions are the problem?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Sorry, I was upthumbing

Not having an ability to chain seems a bit degrading, though users can just use ? operator for that

Meaning I'd be on board with Result<Self>.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I'm fine with changing to Result(). Would you prefer that? Are others on board?

I agree that sounds better to me

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

)
})?;

let offset = ordinal_to_offset.get(&ordinal).ok_or_else(|| {

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

tiny nit, to simplify L68 below?

Suggested change
let offset = ordinal_to_offset.get(&ordinal).ok_or_else(|| {
let offset = *ordinal_to_offset.get(&ordinal).ok_or_else(|| {

@alamb alamb left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Thank you @vustef -- I think this is a nice fix. I also wrote another "end to end" style test (passing in row numbers to the reader options) which I will push directly to this branch

Thank you @scovich for finding this bug before release 🙏

Given the suggestions from @etseidl and @scovich about avoiding panics, which seems reasonable, I will try and make a follow on PR to do that, as I think this one stands alone as is (no reason to also include the API change there too)

Update: PR is

@alamb alamb changed the title Fix RowNumberReader Fix RowNumberReader when not all row groups are selected Nov 18, 2025
@alamb alamb merged commit c45d238 into apache:main Nov 18, 2025
16 checks passed
alamb added a commit that referenced this pull request Nov 19, 2025
…ic on invalid input (#8867)

# Which issue does this PR close?

- Follow on to #8715
- related to #8863



# Rationale for this change

per #8715 (comment),
@scovich rightly says

> not a fan of panics

It is much better for a user facing API to return an error on invalid
input than painc

# What changes are included in this PR?

1. Make `ArrowReaderOptions::with_virtual_columns` error rather than
panic on invalid input
2.  Update tests to match

# Are these changes tested?
Yes by CI


# Are there any user-facing changes?
While this is an API change it was introduced in
#8715 which has not yet been
released. Therefore, we can make this change without breaking semver.
pull Bot pushed a commit to TCeason/arrow-datafusion that referenced this pull request May 28, 2026
…and ParquetOpener (apache#22026)

## Which issue does this PR close?

- Part of apache#20135 (epic: virtual / metadata columns). Does not close that
epic; see [this
comment](apache#20135 (comment))
describing the scope split.
- Revives apache#20133 (auto-closed stale) — same core plumbing, credit to
@jkylling.
- Unblocks apache/datafusion-comet#3432 (remove native_datafusion
fallback for Spark's `_tmp_metadata_row_index`).

## Rationale for this change

arrow-rs 57.1.0+ supports Parquet virtual columns (`row_number`,
`row_group_index`) via `ArrowReaderOptions::with_virtual_columns`, and
DataFusion pins a new-enough arrow-rs for the API to be available.
DataFusion does not yet plumb the option through `ParquetOpener`, so
consumers (notably Comet) cannot project Spark's
`_tmp_metadata_row_index` through the native_datafusion scan path.

This PR adds the minimal opener-boundary plumbing so `TableSchema` can
carry virtual columns and the Parquet reader produces them. UX /
SQL-layer surface for virtual columns stays deferred to the epic in
apache#20135 — this follows the same framing alamb blessed for apache#20071 (the
`input_file_name()` UDF).

## What changes are included in this PR?

### `TableSchema` / `TableSchemaBuilder`

- `TableSchemaBuilder::with_virtual_columns(impl Into<Fields>)` setter,
picking up apache#22496's "follow-up" hook: build with `[file, partition,
virtual]` ordering in a single concatenation. Setter order on the
builder does not matter; the layout is fixed.
- Virtual columns are stored as `arrow::datatypes::Fields` (matches the
`table_partition_cols` storage main switched to in apache#22496 — no
`Arc<Vec<_>>` indirection, shareable zero-copy, immutable so the
in-place mutation panic class is structurally impossible).
- `TableSchema::virtual_columns()` getter (`&Fields`).
- `TableSchema::schema_without_virtual_columns()` — file + partition
schema used by pushdown-planning paths that can't evaluate virtual-col
refs.
- `TableSchema::with_virtual_columns(...)` chainable convenience
preserved for API ergonomics; routes through the builder so it preserves
any partition columns already on the source `TableSchema`.
- The deprecated `TableSchema::with_table_partition_cols` was extended
to preserve virtual columns when routing through the builder (it would
otherwise drop them for callers still on the deprecated path).
- Collision check (`virtual` vs `file`, `virtual` vs `partition`,
duplicates within `virtual`) lives in `TableSchemaBuilder::build()` as a
`debug_assert!` so release builds pay nothing; setter order is
irrelevant because the check runs at `build()`.

### `ParquetOpener` / `ParquetSource`

- `ParquetOpener` forwards the fields to
`ArrowReaderOptions::with_virtual_columns`; augments the schemas passed
to the expr-adapter / simplifier with virtual fields so virtual-col refs
identity-rewrite; the virtual-col stripping (substitute with null
literals for `ProjectionMask::roots`, append to `stream_schema` so
`reassign_expr_columns` resolves them by name) lives inside
`DecoderProjection::try_new` (the abstraction apache#22398 introduced),
reached via a new `Option<&VirtualColumnsState>` parameter so the
zero-virtual-column common path is unchanged.
- New `ParquetVirtualColumn` enum with `TryFrom<&FieldRef>` (in
`datasource-parquet::virtual_column`) gates which arrow-rs virtual
extension types are accepted. Currently only `RowNumber`; adding a
variant (e.g. `RowGroupIndex`) is a compile-time obligation. Replaces a
runtime string-allowlist so the contract lives in the type system.
- `ParquetSource::try_pushdown_filters` classifies filters against
`schema_without_virtual_columns()` so predicates referencing virtual
columns are reported as `PushedDown::No` and the `FilterExec` stays
above the scan — arrow-rs's `RowFilter` addresses parquet leaves only
and can't evaluate virtual-column refs, so silently pushing them would
produce wrong results.
- Defensive check in the opener: `build_virtual_columns_state` (run once
per scan partition at morselizer-build time) errors when
`pushdown_filters=true` and the predicate references a virtual column,
with a clear remediation message pointing at `try_pushdown_filters`.
This catches callers that bypass the optimizer and set the predicate on
`ParquetSource` directly. Returns a `Result` (not a panic) so the
contract is enforced in release builds too.
- `VirtualColumnsState` is constructed once per scan partition:
validates the extension-type allowlist, precomputes the
`null_replacements` HashMap and the `logical_schema_with_virtual`
schema. Each file's open path then borrows the precomputed state via
`Arc`.

### Cargo

- `arrow-schema` added as a direct dep (previously transitive via
`arrow`) so the enum references `RowNumber::NAME` from arrow-rs (via
`arrow_schema::extension::ExtensionType`) instead of hardcoding the
string.

### Explicitly **not** in scope (follow-ups)

- `ListingTable` / SQL-layer surface
- `ParquetSource::with_virtual_columns`
- `RowGroupIndex` support (the enum has a deliberate-rejection test for
it)
- Removing the `TableSchema` chainable convenience setter; it could be
deprecated in a follow-up to align fully with apache#22496's builder-only
direction

## Are these changes tested?

Yes.

**`opener/mod.rs`** (10 new tests, in a `virtual_columns` submodule):

- `test_row_index_basic` — single row group, select data + row_number.
- `test_row_index_projection_only` — select only row_number.
- `test_row_index_multi_row_group` — 3 × 100 rows, verify absolute
0..300 across boundaries.
- `test_row_index_with_row_group_skip` — predicate stats-prunes the
middle row group; verify row numbers stay absolute (0..100 ++ 200..300).
Critical correctness gate for Spark (and for apache/arrow-rs#8863).
- `test_row_index_with_partition_cols` — partition + virtual + data
columns compose correctly.
- `test_row_index_nullable_int64` — nullability flag flows through
unchanged (matches Spark's `_tmp_metadata_row_index` declaration).
- `test_unsupported_virtual_extension_type_rejected` — using
`RowGroupIndex` (a real arrow-rs type deliberately not in the enum yet)
errors with `NotImplemented` instead of silently forwarding.
- `test_row_index_predicate_pushdown_mixed_or_errors` /
`_virtual_only_errors` / `_allowed_when_pushdown_disabled` — exercise
the opener's defensive check for virtual-col predicate refs with
`pushdown_filters=true`, and confirm the `pushdown_filters=false` path
is unaffected.

**`source.rs`**: `test_try_pushdown_filters_rejects_virtual_column_refs`
pins the planner-boundary contract — file-col filters are
`PushedDown::Yes`, virtual-only and mixed `OR` filters are
`PushedDown::No`.

**`virtual_column.rs`** (3 new tests): `TryFrom<&FieldRef>` for valid
`RowNumber`, missing-extension-type, and unsupported-extension-type
(real `RowGroupIndex`) inputs.

**`table_schema.rs`** (5 new tests): `[file, partition, virtual]` layout
regardless of builder-call order; `debug_assert!` collision panics for
virtual-vs-file, virtual-vs-partition (both setter orderings), and
duplicates within virtual.

`cargo test -p datafusion-datasource-parquet --all-features` (137
passing) and `cargo test -p datafusion-datasource` (148 passing). `cargo
clippy -p datafusion-datasource-parquet -p datafusion-datasource
--all-targets --all-features -- -D warnings` is clean.

## Are there any user-facing changes?

Public API additions:

- `TableSchemaBuilder::with_virtual_columns(impl Into<Fields>)`
- `TableSchema::with_virtual_columns(Vec<FieldRef>)` (chainable
convenience that routes through the builder)
- `TableSchema::virtual_columns() -> &Fields`
- `TableSchema::schema_without_virtual_columns() -> SchemaRef`
- `ParquetVirtualColumn` (re-exported from
`datafusion-datasource-parquet`)

No breaking changes; no existing API changed.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

parquet Changes to the parquet crate

Projects

None yet

Development

Successfully merging this pull request may close these issues.

RowNumber reader has wrong row group ordering

4 participants