openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r470399983



##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import org.apache.commons.compress.utils.Lists;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import 
org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class IcebergFilesCommitter extends RichSinkFunction<DataFile> implements
+    CheckpointListener, CheckpointedFunction {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergFilesCommitter.class);
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = 
"flink.max-committed.checkpoint.id";
+
+  private final String path;
+  private final SerializableConfiguration conf;
+
+  private transient long maxCommittedCheckpointId;
+  private transient NavigableMap<Long, List<DataFile>> dataFilesPerCheckpoint;
+  private transient List<DataFile> dataFilesOfCurrentCheckpoint;
+  private transient Table table;
+
+  // State for all checkpoints;
+  private static final ListStateDescriptor<byte[]> STATE_DESCRIPTOR =
+      new ListStateDescriptor<>("checkpoints-state", 
BytePrimitiveArraySerializer.INSTANCE);
+  private transient ListState<byte[]> checkpointsState;
+
+  IcebergFilesCommitter(String path, Configuration newConf) {
+    this.path = path;
+    this.conf = new SerializableConfiguration(newConf);
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws 
Exception {
+    table = TableUtil.findTable(path, conf.get());
+    maxCommittedCheckpointId = 
parseMaxCommittedCheckpointId(table.currentSnapshot());

Review comment:
       You are right.  here we could only initialize the max-checkpoint-id to 
-1,  and could not read the checkpoint id from iceberg table if not in a 
restore job.   Because we may have a table with 3 checkpoints and its 
max-committed-checkpoint id is 3,  then we stop the current flink job and start 
another flink job to continue to write this table,  its checkpoint id will 
start from 1, if we read max-commited-checkpoint id here then we will miss the 
first three checkpoint's data files in current flink job.




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

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