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



##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.SortedMap;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ListTypeInfo;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo;
+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.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class IcebergFilesCommitter extends AbstractStreamOperator<Void>
+    implements OneInputStreamOperator<DataFile, Void>, BoundedOneInput {
+
+  private static final long serialVersionUID = 1L;
+  private static final long INITIAL_CHECKPOINT_ID = -1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergFilesCommitter.class);
+  private static final String FLINK_JOB_ID = "flink.job-id";
+
+  // The max checkpoint id we've committed to iceberg table. As the flink's 
checkpoint is always increasing, so we could
+  // correctly commit all the data files whose checkpoint id is greater than 
the max committed one to iceberg table, for
+  // avoiding committing the same data files twice. This id will be attached 
to iceberg's meta when committing the
+  // iceberg transaction.
+  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 SerializableConfiguration hadoopConf;
+
+  // A sorted map to maintain the completed data files for each pending 
checkpointId (which have not been committed
+  // to iceberg table). We need a sorted map here because there's possible 
that few checkpoints snapshot failed, for
+  // example: the 1st checkpoint have 2 data files <1, <file0, file1>>, the 
2st checkpoint have 1 data files
+  // <2, <file3>>. Snapshot for checkpoint#1 interrupted because of 
network/disk failure etc, while we don't expect
+  // any data loss in iceberg table. So we keep the finished files <1, <file0, 
file1>> in memory and retry to commit
+  // iceberg table when the next checkpoint happen.
+  private final NavigableMap<Long, List<DataFile>> dataFilesPerCheckpoint = 
Maps.newTreeMap();

Review comment:
       Yeah,  I saw that it will archive all the finished data files into a 
manifest file and maintain only the manifest files in state backend. 
(https://github.com/apache/iceberg/pull/856/files#diff-0245903e25b2cace005c52b2c7b130bbR372)
 .   For my understanding, that's a great point to handle the case that we fail 
to commit to iceberg table frequently (because we'v buffered all the data files 
in the manifest files rather than flink state backend),  but I'm no sure 
whether it is a common case for most of users,  the commit failure should be 
easily  found if they separated the data files regions from the metastore 
region,  others also use this way to maitain their data and meta ?  or their 
states  are enough to maintain those uncommitted data files ?  
   
   If the commit failure is a common case,  I'd happy to contribute to make 
those data files buffered in a manifest file.   For now,  I think we can make 
the simple state way go firstly.  Does that make sense ?  




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