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


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/v2/FilesCommitter.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.v2;
+
+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.SnapshotUpdate;
+import org.apache.iceberg.Table;
+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.util.ThreadPools;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class FilesCommitter implements Committer<FilesCommittable>, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(FilesCommitter.class);
+  private static final String FLINK_JOB_ID = "flink.job-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 int continuousEmptyCheckpoints;
+
+  private transient ExecutorService workerPool;
+
+  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 dataFilesNumRestored = 0;
+    int deleteFilesNumRestored = 0;
+
+    int dataFilesNumStored = 0;
+    int deleteFilesNumStored = 0;
+
+    Collection<CommitRequest<FilesCommittable>> restored = 
Lists.newArrayList();
+    Collection<CommitRequest<FilesCommittable>> store = Lists.newArrayList();
+
+    for (CommitRequest<FilesCommittable> request : requests) {
+      WriteResult committable = request.getCommittable().committable();
+      if (request.getNumberOfRetries() > 0) {
+        restored.add(request);
+        dataFilesNumRestored = dataFilesNumRestored + 
committable.dataFiles().length;
+        deleteFilesNumRestored = deleteFilesNumRestored + 
committable.deleteFiles().length;
+      } else {
+        store.add(request);
+        dataFilesNumStored = dataFilesNumStored + 
committable.dataFiles().length;
+        deleteFilesNumStored = deleteFilesNumStored + 
committable.deleteFiles().length;
+      }
+    }
+
+    if (restored.size() > 0) {
+      commitResult(dataFilesNumRestored, deleteFilesNumRestored, restored);
+    }
+
+    commitResult(dataFilesNumStored, deleteFilesNumStored, store);
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (tableLoader != null) {
+      tableLoader.close();
+    }
+
+    if (workerPool != null) {
+      workerPool.shutdown();
+    }
+  }
+
+  private void commitResult(int dataFilesNum, int deleteFilesNum,
+                            Collection<CommitRequest<FilesCommittable>> 
committableCollection) {
+    int totalFiles = dataFilesNum + deleteFilesNum;
+
+    continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints 
+ 1 : 0;
+    if (totalFiles != 0) {
+      if (replacePartitions) {
+        replacePartitions(committableCollection, deleteFilesNum);
+      } else {
+        commitDeltaTxn(committableCollection, dataFilesNum, deleteFilesNum);
+      }
+
+      continuousEmptyCheckpoints = 0;
+    }
+  }
+
+  private void replacePartitions(Collection<CommitRequest<FilesCommittable>> 
writeResults, int deleteFilesNum) {
+    // Partition overwrite does not support delete files.
+    Preconditions.checkState(deleteFilesNum == 0, "Cannot overwrite partitions 
with delete files.");
+
+    // Commit the overwrite transaction.
+    ReplacePartitions dynamicOverwrite = 
table.newReplacePartitions().scanManifestsWith(workerPool);
+
+    int numFiles = 0;
+    String flinkJobId = "";
+    for (CommitRequest<FilesCommittable> writeResult : writeResults) {
+      WriteResult result = writeResult.getCommittable().committable();
+      Preconditions.checkState(result.referencedDataFiles().length == 0, 
"Should have no referenced data files.");
+      flinkJobId = writeResult.getCommittable().jobID();
+      numFiles += result.dataFiles().length;
+      Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile);
+    }
+
+    commitOperation(dynamicOverwrite, numFiles, 0, "dynamic partition 
overwrite", flinkJobId);
+  }
+
+  private void commitDeltaTxn(Collection<CommitRequest<FilesCommittable>> 
writeResults, int dataFilesNum,
+                              int deleteFilesNum) {
+    if (deleteFilesNum == 0) {
+      // To be compatible with iceberg format V1.
+      AppendFiles appendFiles = 
table.newAppend().scanManifestsWith(workerPool);
+      String flinkJobId = "";
+      for (CommitRequest<FilesCommittable> commitRequest : writeResults) {
+        try {
+          WriteResult result = commitRequest.getCommittable().committable();
+          Preconditions.checkState(
+              result.referencedDataFiles().length == 0,
+              "Should have no referenced data files.");
+
+          Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile);
+          flinkJobId = commitRequest.getCommittable().jobID();
+        } catch (Exception e) {
+          commitRequest.signalFailedWithUnknownReason(e);
+        }
+      }
+
+      commitOperation(appendFiles, dataFilesNum, 0, "append", flinkJobId);
+      writeResults.forEach(CommitRequest::signalAlreadyCommitted);

Review Comment:
   I have tested the following sequence (slightly modified 
TestIcebergInputFormats.testFilterExp):
   ```java
   [..]
       helper = new TestHelper(conf, tables, location.toString(), SCHEMA, SPEC, 
fileFormat, temp);
   [..]
       helper.createTable();
   
       List<Record> expectedRecords = helper.generateRandomRecords(2, 0L);
       expectedRecords.get(0).set(2, "2020-03-20");
       expectedRecords.get(1).set(2, "2020-03-20");
   
       DataFile dataFile1 = helper.writeFile(Row.of("2020-03-20", 0), 
expectedRecords);
       DataFile dataFile2 = helper.writeFile(Row.of("2020-03-21", 0), 
helper.generateRandomRecords(2, 0L));
       helper.appendToTable(dataFile1, dataFile2); // This creates a 
transaction and adds the data files to it using 'table.newAppend()'
       helper.appendToTable(dataFile1, dataFile2);
   ```
   
   The resulting table contained the expected records twice. So the Iceberg 
table commit is not idempotent, adding the same file twice will duplicate the 
record. For me this means that we have to make sure that the 
`FilesCommitter.commit` is idempotent ourselves.
   
   Reading only the FlinkSink documentation, I think that calling 
`commitRequest.signalAlreadyCommitted` does not guarantee that the commit 
operation will not be called again. I think we can not rely on any of the Flink 
variables / objects to know if a commit happened or not. I think we have to 
deduce that from reading the Iceberg table data / metadata for the current 
snapshot, as our only reliable source of information.
   
   @hililiwei: Does this make sense? Do I miss something?
   
   Thanks,
   Peter



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