pvary commented on code in PR #14810:
URL: https://github.com/apache/iceberg/pull/14810#discussion_r2613353949
##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java:
##########
@@ -126,9 +125,14 @@ public void
commit(Collection<CommitRequest<DynamicCommittable>> commitRequests)
return;
}
- // For every table and every checkpoint, we store the list of
to-be-committed
- // DynamicCommittable.
- // There may be DynamicCommittable from previous checkpoints which have
not been committed yet.
+ /*
+ Each (table, branch, checkpoint) triplet must have only one commit
request.
+ There may be commit requests from previous checkpoints which have not
been committed yet.
+
+ We currently keep a List of commit requests per checkpoint instead of a
single CommitRequest<DynamicCommittable>
+ to process the Flink state from previous releases, which had multiple
commit requests due to a bug in the upstream
+ DynamicWriteResultAggregator. We should replace this with a single
commit request in the next major release.
Review Comment:
With the old Sink we had the following guarantees:
- You were able to upgrade between Iceberg minor versions without any issue
- Flink version upgrade needed an application stop-start. In this case the
committable state was cleaned (notifyCheckpointComplete was run), and these
incompatibilities were not an issue.
With the new SinkV2 API, I'm not sure we have any guarantees for cleaning up
the committables from the state, but if this is so, then we might need to keep
these things for a longer period.
--
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]