aiborodin opened a new issue, #14090:
URL: https://github.com/apache/iceberg/issues/14090

   ### Apache Iceberg version
   
   1.10.0 (latest release)
   
   ### Query engine
   
   Flink
   
   ### Please describe the bug 🐞
   
   `DynamicWriteResultAggregator` in `DynamicIcebrgSink` currently produces 
multiple dynamic committables per table/branch/checkpoint triplet because it 
aggregates write results by `WriteTarget`, which is unique per schemaId, 
specId, and equality fields. It violates the idempotence contract of the 
`DynamicCommitter`, which
   relies on one commit request per triplet to identify and skip already 
committed requests during recovery.
   
   For example, a data loss occurs in the following scenario:
   1. Sink creates two commit requests with properties `CheckpointID = 1`, 
`JobID = a`, `OperatorID = abc`:
   - `Commit 1` (data with Schema 1)
   - `Commit 2` (data with Schema 2)
   2. Commit requests are saved to a checkpoint
   3. Committer commits `Commit 1`
   4. Flink job restarts due to a commit failure or an autoscaling event
   5. Commit requests are retrieved from a checkpoint
   6. `DynamicCommitter` checks the Iceberg snapshots and identifies that the 
`Committable 1` has already been committed as it matches `CheckpointID = 1`, 
`JobID = a`, `OperatorID = abc`
   - `DynamicCommitter` skips the subsequent `Committable 2`, which is part of 
the same checkpoint
   
   ### Willingness to contribute
   
   - [x] I can contribute a fix for this bug independently
   - [ ] I would be willing to contribute a fix for this bug with guidance from 
the Iceberg community
   - [ ] I cannot contribute a fix for this bug at this time


-- 
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: issues-unsubscr...@iceberg.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to