pvary commented on code in PR #14182:
URL: https://github.com/apache/iceberg/pull/14182#discussion_r2381722320
##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java:
##########
@@ -274,26 +274,25 @@ private void replacePartitions(
CommitSummary summary,
String newFlinkJobId,
String operatorId) {
- for (Map.Entry<Long, List<WriteResult>> e : pendingResults.entrySet()) {
- // We don't commit the merged result into a single transaction because
for the sequential
- // transaction txn1 and txn2, the equality-delete files of txn2 are
required to be applied
- // to data files from txn1. Committing the merged one will lead to the
incorrect delete
- // semantic.
- for (WriteResult result : e.getValue()) {
- ReplacePartitions dynamicOverwrite =
- table.newReplacePartitions().scanManifestsWith(workerPool);
+ // Iceberg tables are unsorted. So the order of the append data does not
matter.
+ // Hence, we commit everything in one snapshot.
+ ReplacePartitions dynamicOverwrite =
table.newReplacePartitions().scanManifestsWith(workerPool);
+
+ for (List<WriteResult> writeResults : pendingResults.values()) {
+ for (WriteResult result : writeResults) {
Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile);
- commitOperation(
- table,
- branch,
- dynamicOverwrite,
- summary,
- "dynamic partition overwrite",
- newFlinkJobId,
- operatorId,
- e.getKey());
}
}
+
+ commitOperation(
+ table,
+ branch,
+ dynamicOverwrite,
+ summary,
+ "dynamic partition overwrite",
+ newFlinkJobId,
+ operatorId,
+ pendingResults.lastKey());
Review Comment:
We still need to commit the checkpoints one-by-one. What if the replace
happened for the same partition? With the proposed method we will have
duplicated data
--
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]