feat(2152): support pa.RecordBatchReader in Table.append/overwrite#3335
Open
paultmathew wants to merge 2 commits intoapache:mainfrom
Open
feat(2152): support pa.RecordBatchReader in Table.append/overwrite#3335paultmathew wants to merge 2 commits intoapache:mainfrom
paultmathew wants to merge 2 commits intoapache:mainfrom
Conversation
Currently `Table.append(df)` and `Table.overwrite(df)` only accept a materialised `pa.Table`, which forces callers to load the entire dataset into memory before writing. This makes pyiceberg unusable for large or unbounded inputs and has been a recurring complaint (apache#1004, apache#2152, dlt-hub#3753). Allow `pa.RecordBatchReader` as an alternative input. When a reader is provided, batches are streamed and microbatched into target-sized Parquet files via the new `bin_pack_record_batches` helper, then committed in a single snapshot via the existing fast_append path. Memory is bounded by `write.target-file-size-bytes` (default 512 MiB) per worker rather than the full input size. Scope of this PR — unpartitioned tables only. Streaming into partitioned tables raises NotImplementedError pointing back to apache#2152; partitioned support needs additional design (high-cardinality partition handling, per-partition rolling writers) and is tracked as a follow-up. Mirrors iceberg-go#369's staging — that project shipped unpartitioned streaming first. Internal note: the implementation buffers up to `target_file_size` of in- memory RecordBatches before flushing to a Parquet file. A more memory- efficient rolling-ParquetWriter approach is a planned follow-up that will benefit from the `OutputStream.tell()` API added in apache#2998.
4ecaf15 to
67c64ec
Compare
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Rationale for this change
Closes #2152, addresses the long-standing memory problem reported in #1004 and re-discovered by dlt-hub#3753.
Table.append(df)andTable.overwrite(df)currently require a fully materialisedpa.Table. For large or unbounded inputs this means loading the entire dataset into memory before writing — fatal at any non-trivial scale and a recurring complaint going back to #1004 (Aug 2024). The reference Java implementation has streaming append; iceberg-go shipped it in iceberg-go#369 (Apr 2025). Python is the last major SDK without it.This PR adds
pa.RecordBatchReaderas a valid input toTable.append/overwrite(andTransaction.append/overwrite). The reader is consumed lazily, microbatched into Parquet files via the newbin_pack_record_batcheshelper, and committed in a single snapshot via the existingfast_appendpipeline.Scope (unpartitioned only)
Streaming into a partitioned table raises
NotImplementedErrorpointing back to #2152. Partitioned support is genuinely the harder case — it needs design discussion around partition cardinality bounds, per-partition rolling writers, and idempotency on retry — so I'm proposing to land in three reviewable PRs:pq.ParquetWriter+OutputStream.tell()for constant-memory streaming. No API change. Detailed plan below.This mirrors iceberg-go#369's staging: ship the unpartitioned API first, iterate from there.
Implementation
The streaming path reuses the existing
WriteTask→write_file→fast_appendpipeline. The only new primitive isbin_pack_record_batches(sibling of the existingbin_pack_arrow_table):RecordBatches into an in-memory buffer.sum(batch.nbytes) >= write.target-file-size-bytes.write_parquettask._check_pyarrow_schema_compatible) runs againstreader.schemabefore the snapshot producer opens — schema mismatches fail before any data file is written, so no orphans.Acknowledged trade-offs
Memory: peak memory is bounded by
N_workers × write.target-file-size-bytes(default 8 × 512 MiB ≈ 4 GiB), not constant. This is materially better than today's "materialise everything" but isn't yet "constant memory streaming". PR2 fixes this.Byte semantics:
write.target-file-size-bytesis currently interpreted as uncompressed in-memory Arrow bytes (RecordBatch.nbytes— the bin-packing weight), not compressed on-disk Parquet bytes. The resulting files are typically 3-10× smaller than the property suggests after zstd / dictionary / RLE encoding. This matches the existingpa.Tablewrite path (bin_pack_arrow_tableuses the same accounting) — this PR doesn't change pyiceberg's existing semantics, it only documents them in the docstrings of both helpers and theTransaction.append/overwriteNote:blocks. PR2 fixes this too.Retry:
pa.RecordBatchReaderis single-pass, so a failed catalog commit leaves the reader drained and a naive retry writes zero rows. Documented in theNote:block — callers needing at-least-once semantics should reconstruct the reader on each attempt via a factory callable, or use the two-stageadd_filespattern (whose input is a replayable list of paths).PR2 — proposed scope (FYI, not in this PR)
Drop the buffer-and-flush approach and use a rolling
pq.ParquetWriterdriven byOutputStream.tell()(added in #2998 specifically for this kind of use case):What this delivers:
O(1 batch)per worker (~10s of MB) regardless oftarget_file_size. The 4 GiB peak in this PR drops to ~50-100 MB.write.target-file-size-bytesbecomes actual on-disk compressed bytes, matching the Java/Spark/Flink writers and the spec.tx.append(reader)/tx.overwrite(reader)— internals only.Open design questions for PR2 (will surface on the issue thread before coding):
executor.map(write_parquet, tasks)parallelism.bin_pack_arrow_tableto the same rolling-writer mechanism would also tighten thepa.Tablepath's byte semantics. That changes file-size characteristics for every existing pyiceberg writer. Probably worth a separate change with a deprecation note, or a feature flag.add_filesinteraction: rolling writes produce data files we know about directly; we shouldn't go through the parquet-footer round-trip in_dataframe_to_data_files. Means a small refactor in the streaming-only path.Are these changes tested?
Yes, comprehensively at four layers.
1. Unit tests (
tests/io/test_pyarrow.py) — 4 new tests forbin_pack_record_batchescovering single-bin, microbatched, empty input, and lazy generator consumption.2. End-to-end behaviour tests (
tests/catalog/test_catalog_behaviors.py) — 8 new tests parametrised across all three in-process catalog backends (memory,sql,sql_without_rowcount) → 24 test runs covering append, overwrite, microbatch verification (multiple files in one snapshot), empty reader, partitioned-table-raises, invalid-input-rejected, reader-consumed-exactly-once, and schema-mismatch-writes-no-files.3. Integration tests (
tests/integration/test_writes/test_writes.py) — 6 new Spark-readback tests for v1 + v2 format versions covering append, overwrite, and multi-file microbatch. Proves Spark can read tables written via the streaming path against the docker-compose stack.4. Smoke test on a real production stack — verified end-to-end against AWS Glue + S3 in our staging account: 100 k-row streaming append in 17 s, 20-file microbatched commit, Athena read-back (
COUNT(*)andMAX(id)matched the input exactly), schema-mismatch rejection leaving no orphan files.Full unit suite: 3 647 passed. Full integration suite: 122 passed, 1 skipped.
Are there any user-facing changes?
Yes, intentionally:
Transaction.append(df),Transaction.overwrite(df),Table.append(df),Table.overwrite(df)acceptpa.Table | pa.RecordBatchReader.ValueErrorraised on bad input changes from"Expected PyArrow table, got: ..."to"Expected pa.Table or pa.RecordBatchReader, got: ...". Updatedtest_invalid_argumentsaccordingly.bin_pack_record_batchesinpyiceberg.io.pyarrow(sibling ofbin_pack_arrow_table).bin_pack_arrow_tablegained its first docstring, documenting the existing uncompressed-Arrow-bytes accounting.mkdocs/docs/api.md.Transaction.append/overwritedocument retry semantics and the byte-semantics caveat.Related
write_parquetAPI for writing Parquet files without committing #1742's review (_dataframe_to_data_files+fast_append.append_data_file(), no separatewrite_parquetAPI)test_write_optional_listOutputStream.tell()from feat: Add tell() to OutputStream writers #2998