-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Managed Iceberg] unbounded source #33504
base: master
Are you sure you want to change the base?
Conversation
…erg_streaming_source
…erg_streaming_source
Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment |
R: @kennknowles Can y'all take a look? I still have to write some tests, but it's at a good spot for a first round of reviews. I ran a bunch of pipelines (w/Legacy DataflowRunner) at different scales and the throughput/scalability looks good. |
Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control. If you'd like to restart, comment |
…erg_streaming_source
…erg_streaming_source
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Overall, I think all the pieces are in the right place. Just a question about why an SDF is the way it is and a couple code-level comments.
This seems like something you want to test a lot of different ways before it gets into a release. Maybe get another set of eyes like @chamikaramj or @Abacn too. But I'm approving and leaving to your judgment.
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTask.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTaskDescriptor.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromGroupedTasks.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Wait actually I forgot I want to have the discussion about the high level toggle between incremental scan source and bounded source.
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromGroupedTasks.java
Outdated
Show resolved
Hide resolved
…erg_streaming_source
…rk progress; convert GiB output iterable to list because of RunnerV2 bug
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Show resolved
Hide resolved
…ng' option; doc updates
…erg_streaming_source
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks!
sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java
Show resolved
Hide resolved
@chamikaramj this is ready for another review |
…erg_streaming_source
.discardingFiredPanes()) | ||
.apply( | ||
GroupIntoBatches.<ReadTaskDescriptor, ReadTask>ofByteSize( | ||
MAX_FILES_BATCH_BYTE_SIZE, ReadTask::getByteSize) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we don't really want these batches, we just want the read tasks distributed to workers without causing worker ooms. Otherwise we're just adding latency for the poll latency and not really benefitting from the batch.
Ideally we could change Redistribute to autoshard, but since it is tied to GroupIntoBatches currently, what about just doing GroupIntoBatches.ofSize(1).withShardedKey() ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I initially figured that GroupIntoBatches.ofSize(1).withShardedKey()
would give us too many concurrent shards, but after running I found it actually produces only 1 shard, and everything is processed sequentially. Same thing when I tried .ofByteSize(1)
GroupIntoBatches.ofSize(1).withShardedKey().withMaxBufferingDuration(pollInterval)
: 2025-03-07_08_57_21-15760437490773458424GroupIntoBatches.ofByteSize(1).withShardedKey().withMaxBufferingDuration(pollInterval)
: 2025-03-07_09_04_50-7891042636475112191
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I also tried mimicking GiB behavior by:
- Associating a key to each read task. The key is incremented after reaching 4GB.
- Adding a Redistribute.byKey() after CreateReadTasksDoFn to distribute read tasks into 4GB per streaming key
Read-and-drop did fine but the throughput was pretty spiky. Read + write took much longer than the current approach and didn't scale well: 2025-03-07_08_57_21-15760437490773458424
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java
Outdated
Show resolved
Hide resolved
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java
Show resolved
Hide resolved
|
||
return isComplete | ||
? PollResult.complete(timestampedSnapshots) // stop at specified snapshot | ||
: PollResult.incomplete(timestampedSnapshots); // continue forever |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we want to generate a correct watermark here using
PollResult.withWatermark
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added watermark to the TimestampedValue above, as well as to individual read tasks outputted by CreateReadTasksDofn
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. LGTM.
* <td> {@code operation} </td> | ||
* <td> {@code string} </td> | ||
* <td> | ||
* The snapshot <a href="https://iceberg.apache.org/javadoc/0.11.0/org/apache/iceberg/DataOperations">operation</a> associated with this record. For now, only "append" is supported. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
May be change to "APPEND" to be consistent with Iceberg.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The value is actually lowercase (see ref)
* | ||
* <p><b>Note</b>: This reads <b>append-only</b> snapshots. Full CDC is not supported yet. | ||
* | ||
* <p>The CDC <b>streaming</b> source (enabled with {@code streaming=true}) continuously polls the |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should validate (and fail) somewhere if the "streaming" flag is set here and the streaming PipelineOption [1] is not set.
[1]
beam/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java
Line 38 in c1d0fa4
void setStreaming(boolean value); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done (although note that this is automatically true if there's an unbounded PCollection)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually the flag isn't automatically applied with DirectRunner. I'd prefer to remove this check because it makes this less usable, it forces users to do something like pipeline.getOptions().as(StreamingOptions.class).setStreaming(true);
. We also don't have any precedent for enforcing this on our current unbounded sources.
@@ -108,6 +110,7 @@ public class Managed { | |||
* | |||
* <ul> | |||
* <li>{@link Managed#ICEBERG} : Read from Apache Iceberg tables | |||
* <li>{@link Managed#ICEBERG_CDC} : CDC Read from Apache Iceberg tables |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should link to locations where users can find additional Javadocs related to each of these options (also for write).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
…remove window step; add --strea ming=true validation; add IO links to Managed java doc
Unbounded (streaming) source for Managed Iceberg.
See design doc for high level overview: https://s.apache.org/beam-iceberg-incremental-source
Fixes #33092