paultmathew commented on issue #2152:
URL:
https://github.com/apache/iceberg-python/issues/2152#issuecomment-4400940971
Quick update on the streaming write work:
- **PR1 — #3335** is up (ready for review). `Transaction.append/overwrite`
accept `pa.RecordBatchReader`, microbatched via the buffered
`bin_pack_record_batches` helper. Unpartitioned only.
- **PR2 — #3336** is up as a draft, stacked on PR1. Replaces the buffered
approach with a rolling `pq.ParquetWriter` driven by `OutputStream.tell()`
(#2998). Two concrete wins over PR1:
- **Spec-correct file sizes**: `write.target-file-size-bytes` now reflects
on-disk compressed bytes, matching Java/Spark/Flink. Closes the proxy-bytes
caveat documented in PR1.
- **70× lower peak memory at default settings**: smoke-tested on AWS
Glue+S3 streaming a 515 MiB workload across 24 files — peak RSS 236 MiB, no
growth from start to finish (chart in the PR description). PR1's bound was
`N_workers × target_file_size` ≈ 4 GiB.
- Cross-engine readback verified: pyiceberg own scan, Spark v1, Spark v2,
Athena via Glue catalog all round-trip cleanly.
- **PR3 — partitioned streaming** is still the open design question. I'd
love thread input on:
- **Partition cardinality**: streaming reader with high-cardinality
partition columns implies many concurrent writers. Bound it via spill,
sort-then-stream, or pushdown to caller? iceberg-go #369 punted on this and
partitioned streaming hasn't followed there yet either.
- **Crash/retry idempotency**: per-partition rolling writes commit
metadata only at end-of-stream; partial crashes leave orphans. Today's
`pa.Table` path is naturally transactional because all files are written before
commit. The streaming path inverts that — worth being explicit about whether we
accept orphan-data risk in exchange for streaming, or build cleanup into the
write path.
Will pull #3336 out of draft once #3335 lands. Happy to take feedback on
either PR's approach before that.
--
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]