aiborodin commented on issue #14425: URL: https://github.com/apache/iceberg/issues/14425#issuecomment-3455104523
@pvary I updated the issue description and the diagram with exact details on how this race condition occurs. > The Flink job should fail, and then it should find the new commit on the table history, so it should skip committing the change The Flink job does not fail. REST catalog returns a 409 (`CommitFailedException`) if the base snapshot changes and the [SnapshotProducer](https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/SnapshotProducer.java#L437) retries the table update with a new base snapshot, which creates a duplicate table update request (POST). However, this is a different issue and fixing it would not solve the problem described below. The data duplication happens when the update request 1 completes in the REST catalog after `getMaxCommittedCheckpointId()` check in `DynamicCommitter`, but before the refresh of the base snapshot in `SnapshotProducer.apply()` (illustrated in the diagram). This creates a duplicate table update request 2 with the latest base snapshot (created by the first request), which gets committed instantly with no retries. I think the only way to resolve this problem is to transactionally update both the current metadata pointer and the max committed checkpoint ID (idempotency key) in the underlying catalog. This would prevent the table update request 2 from getting committed because it would fail to atomically swap the latest checkpoint id, which would be already be updated by request 1. For example, in the `JdbcCatalog`, this can be implemented as an update of the latest checkpoint id column or a table property in a database. Iceberg doesn't support transactional updates of catalog properties bundled with a commit request, and implementing this as a generic API sounds like a lot of work. What do you think? -- 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]
