pvary commented on code in PR #4904:
URL: https://github.com/apache/iceberg/pull/4904#discussion_r938469393


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/committer/FilesCommitter.java:
##########
@@ -0,0 +1,314 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink.committer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import org.apache.flink.api.connector.sink2.Committer;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.ReplacePartitions;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.ThreadPools;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FilesCommitter implements Committer<FilesCommittable>, 
Serializable {
+  private static final long serialVersionUID = 1L;
+  private static final long INITIAL_CHECKPOINT_ID = -1L;
+  private static final Logger LOG = 
LoggerFactory.getLogger(FilesCommitter.class);
+  private static final String FLINK_JOB_ID = "flink.job-id";
+  private static final int MAX_RECOMMIT_TIMES = 3;
+
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = 
"flink.max-committed-checkpoint-id";
+
+  // TableLoader to load iceberg table lazily.
+  private final TableLoader tableLoader;
+  private final boolean replacePartitions;
+  private final Map<String, String> snapshotProperties;
+
+  // It will have an unique identifier for one job.
+  private final Table table;
+  private transient Map<String, Long> maxCommittedCheckpointIdForJob = 
Maps.newHashMap();
+
+  private transient ExecutorService workerPool;
+
+  public FilesCommitter(
+      TableLoader tableLoader,
+      boolean replacePartitions,
+      Map<String, String> snapshotProperties,
+      int workerPoolSize) {
+    this.tableLoader = tableLoader;
+    this.replacePartitions = replacePartitions;
+    this.snapshotProperties = snapshotProperties;
+
+    this.workerPool =
+        ThreadPools.newWorkerPool(
+            "iceberg-worker-pool-" + Thread.currentThread().getName(), 
workerPoolSize);
+
+    // Open the table loader and load the table.
+    this.tableLoader.open();
+    this.table = tableLoader.loadTable();
+  }
+
+  @Override
+  public void commit(Collection<CommitRequest<FilesCommittable>> requests)
+      throws IOException, InterruptedException {
+    int dataFilesNumStored = 0;
+    int deleteFilesNumStored = 0;
+
+    Collection<CommitRequest<FilesCommittable>> store = Lists.newArrayList();
+    String jobId = null;
+    long maxCommittableCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    for (CommitRequest<FilesCommittable> request : requests) {
+      WriteResult committable = request.getCommittable().committable();
+      Long committableCheckpointId = request.getCommittable().checkpointId();
+      jobId = request.getCommittable().jobID();
+      if (request.getNumberOfRetries() > MAX_RECOMMIT_TIMES) {
+        String message =
+            String.format(
+                "Failed to commit transaction %s after retrying %d times",
+                request.getCommittable(), MAX_RECOMMIT_TIMES);
+        request.signalFailedWithUnknownReason(new RuntimeException(message));
+      } else if (request.getNumberOfRetries() > 0) {
+        commitResult(
+            committable.dataFiles().length,
+            committable.deleteFiles().length,
+            Lists.newArrayList(request));
+      } else if (committableCheckpointId
+              > maxCommittedCheckpointIdForJob.getOrDefault(jobId, 
Long.MAX_VALUE)
+          || getMaxCommittedCheckpointId(table, jobId) == -1) {

Review Comment:
   Is it possible to commit some files in the pre-job, and commit some files in 
the actual job?
   
   My previous understanding was, that we commit only in the committer, and we 
just collect all of the written files, calculate the min and the max commitId, 
check that it is not in conflict with the current Iceberg table data, and if 
everything is ok, then we commit them in one Iceberg commit.
   
   Reading your code I had to change my mental model a bit, because I 
understand now, that commits with deletes should be committed separately, so we 
need to order the commits by commitId, and collect the commits until we find a 
commit with delete files. In this case, we have to commit the previous batch, 
and then commit to current one, and then we can continue going through the rest.
   
   The other part of my review comment was referring to the ways how Iceberg 
handles commits atomicity and retries. Iceberg detects conflicts. When we start 
an Iceberg transaction we lock on an Iceberg snapshot (the actual snapshot of 
the table object we use to create the change), and if the table is changed 
before the Iceberg commit call, we will see a different current snapshotId for 
the table, and have an error.
   So that is why we have to be careful when we refresh the table and when we 
retrieve the maxCommitId.
   Also there are ways for Iceberg where the code tries to automatically 
recover conflicts. If the files does not conflict, then it tries to commit the 
changes with new metadata without notifying the caller (at least I have read 
about this somewhere). This retry logic might not be desirable for us. We have 
to check if this conflict handling will cause issues with our maxCommitId 
handing, or not



-- 
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]

Reply via email to