Skip to content

rfc: Implement an API for all Data File Formats#2384

Open
Kurtiscwright wants to merge 1 commit intoapache:mainfrom
Kurtiscwright:RFC-Doc-File-Format-API
Open

rfc: Implement an API for all Data File Formats#2384
Kurtiscwright wants to merge 1 commit intoapache:mainfrom
Kurtiscwright:RFC-Doc-File-Format-API

Conversation

@Kurtiscwright
Copy link
Copy Markdown
Contributor

Which issue does this PR close?

What changes are included in this PR?

  • Adds new File Format API RFC doc.
  • From the Issue:
    This RFC proposes a File Format API for the iceberg Rust crate that decouples Iceberg's read and write paths from any single file format. Today, iceberg-rust can only read and write Parquet data files: the format is hard-wired into ArrowReader, ParquetWriter, and every layer that touches them. The Java project shipped an analogous abstraction (FormatModel) in February 2026 via PR Core, Data: File Format API interfaces iceberg#12774, and PyIceberg has an open proposal (File Format API for PyIceberg iceberg-python#3100) for the same concept.

Are these changes tested?

  • Just a doc update.


4. **Redesign the writer trait hierarchy.** The existing `IcebergWriter` and `FileWriter` layering is sound. This RFC adds a format abstraction beneath `FileWriter`, not a replacement for it.

5. **Implement variant shredding or encryption.** Java exposes `engineProjection` and `engineSchema` as extension points for variant shredding and similar format-specific type mapping, and `withFileEncryptionKey` and `withAADPrefix` for Parquet encryption. Equivalent hooks are noted as future extensions in the Rust design. Implementing either requires a dedicated RFC.
Copy link
Copy Markdown
Contributor

@xanderbailey xanderbailey Apr 29, 2026

Choose a reason for hiding this comment

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

Encryption work is currently in-flight so I'd love to understand how we could incorporate that in this plan.

  let mut wb = registry.write_builder(format, output)?;
  if let Some(em) = &self.encryption_manager {
      let (dek, aad) = em.create_file_key().await?;
      wb.with_file_encryption_key(&dek);
      wb.with_aad_prefix(&aad);
  }
  let writer = wb.build().await?;

I need to think if something like this might make sense?

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.

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.

Thank you for the review. In regards to integrating the File Format and Encryption interface, the first thing that jumps out to me is the following question. Does Encryption have different traits or impls based on the File Format in question?

Essentially, will you need OrcEncryption that is separate from ParquetEncryption modules. If yes then later in the doc when it lists the Format specific traits living in the formats/ folder. I think the format specific encryption trait can be called from there. Otherwise we can integrate the Encryption libraries directly into the WriterBuilder and ReaderBuilder with respect to Encrypt vs Decrypt calls.

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.

ORC does have a similar concept to PME in parquet and the spec references this but it's not implemented in Java. In the case of Parquet, the actual encryption / decryption is handled by arrow-rs in the writer so it will differ depending on format. I would hope we could get away with just passing the key and aad down to the writer but I'm the kind of person that has to see code to work out how this might look.

Copy link
Copy Markdown
Collaborator

@CTTY CTTY left a comment

Choose a reason for hiding this comment

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

Hi @Kurtiscwright , thanks for the rfc! Just took a quick look and left some comments

My biggest concern is limiting the format API to the arrow ecosystem. It seems ok for now but it basically forces users to think about arrow when implementing their own formats.

Curious to hear thoughts from you and others as well


2. Remove hard-coded Parquet assumptions from scan and write orchestration. After this work, `TableScan::to_arrow` and `DataFileWriterBuilder` dispatch through the format abstraction instead of constructing Parquet types directly.

3. Provide a registry that maps `DataFileFormat` values to `FormatModel` implementations, so callers obtain readers and writers without naming the concrete format type.
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

I think this is more of an implementation details


3. Provide a registry that maps `DataFileFormat` values to `FormatModel` implementations, so callers obtain readers and writers without naming the concrete format type.

4. Define a conformance test suite (TCK) that any `FormatModel` implementation must pass before it merges.
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

Is this really necessary for the initial implementation? we only support parquet for now

TCK is not even completed on the java side afaik


Each implementation registers one instance per `DataFileFormat` variant it supports. The `format` method returns that variant. `read_builder` and `write_builder` are the entry points for reading and writing a file. Both return trait objects so that the registry can hand them back from a `DataFileFormat`-keyed lookup.

The data type is fixed to Arrow `RecordBatch`, the Iceberg schema type to `iceberg::spec::Schema`, and the physical schema type to `arrow::datatypes::SchemaRef`. These are the only types in iceberg-rust today that would fill the roles Java uses generic parameters for. Arguments for keeping them fixed, including comparisons with Java's `FormatModel<D, S>`, are in "Design rationale" below.
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

We don't have to hardcode it to use Arrow's RecordBatch even. We can use a generic type for in-memory representation and arrow can be the default value


Using `DataFileFormat` as a `HashMap` key requires adding `#[derive(Hash)]` to the enum. That is a non-breaking addition.

The registry is an owned value, not a global static. Tests construct their own. Applications construct one at startup and pass it to scan planners and write orchestrators. For the common case of a single registry for the lifetime of a process, `default_format_registry()` returns a `&'static FormatRegistry` initialized through `OnceLock` on first call.
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

Would love to see more details on scan planner's API changes. Also what about writing?

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

Or we can inject format registry to Catalog instances directly?


`read_builder` and `write_builder` return `Err(Error { kind: ErrorKind::FeatureUnsupported, .. })` for unregistered formats. The error message distinguishes two cases: the format is implemented but its feature flag is disabled in this build, or the format has no implementation in this crate.

### Feature flags
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

This could be a non-goal, we only plan to support parquet for now and parquet is essential

crates/iceberg/src/formats/
├── mod.rs # FormatModel, FormatReadBuilder, FormatWriteBuilder, FormatFileWriter
├── registry.rs # FormatRegistry, default_format_registry
└── parquet.rs # ParquetFormatModel, wrapping existing ParquetWriter and ArrowReader
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

nit: This should be formats/parquet/mod.rs?


Java's `FormatModel<D, S>` uses `D` for Spark `InternalRow`, Flink `RowData`, Arrow `ColumnarBatch`, and other engine-native row types. iceberg-rust has one row type: Arrow `RecordBatch`. Every writer accepts it, and every reader returns it. No format on the near-term queue (ORC, Avro data-file, Vortex, Lance) produces anything other than `RecordBatch` at the Iceberg-facing boundary. No engine integration in iceberg-rust today brings an engine-native row type the way Spark and Flink do in Java.

Adding a `D` parameter today means writing `<RecordBatch>` everywhere the trait is used, for no present caller benefit. If a future format cannot bridge to Arrow, the trait can gain an associated type with a default, which is a semver-compatible addition.
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

the trait can gain an associated type with a default

I think we can do it in the initial cut, but I have not investigated the required effort for this


Feature flags work across `cargo check`, `cargo miri`, cross-compilation, and static linking. `datafusion`, `opendal`, and `reqwest` use the same pattern.

#### RecordBatch as the canonical data type
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

Is this the same as the first point?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

RFC: File Format API for Apache Iceberg Rust

3 participants