You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/07/09 09:10:11 UTC

[GitHub] [iceberg] openinx opened a new pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

openinx opened a new pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479545979



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);

Review comment:
       We might have a problem if redeploying the Flink job from external checkpoint. It is a new `flinkJobId` in this case. maxCommittedCheckpointId will be `-1`.  As a result, we can commit those committed files again later.
   
   The way we do de-dup is to generate a hash for the manifest file path and store the hash in snapshot summary. During restore, we use the hash to de-dup if the manifest file was committed 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478817173



##########
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:
       +1 for this approach. There may be thousands of datafiles in a checkpoint for partitioned table.
   - In `snapshotState`, `serialize files to Flink state` VS. `serialize files to manifest file`, they should cost about the same.
   - In `notifyCheckpointComplete`, just commit a manifest file has better performance instead of commit data files.
   - Can reduce the size of States, which are actually put into the memory of the job manager.
   
   Although it is rare that checkpoint fails for a long time, it can improve the robustness of the system.
   
   But, this approach does lead to more code complexity.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r480564867



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {

Review comment:
       Yes. You don't need care about thread safety.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478774079



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    tableLoader.open(hadoopConf.get());
+    table = tableLoader.loadTable();
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;

Review comment:
       Minor: I prefer to prefix field names with `this.` when setting instance fields, so it is obvious that it is not a local variable.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r480244636



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {

Review comment:
       thx. should we raise a jira to track the task?
   
   @JingsongLi you are saying that notifyCheckpointComplete and snapshotState are serialized by the same lock/mutex, right? Otherwise, I can see problem with the concurrent checkpoint handling.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r473716952



##########
File path: flink/src/main/java/org/apache/iceberg/flink/RowDataTaskWriterFactory.java
##########
@@ -113,7 +114,7 @@ protected PartitionKey partition(RowData row) {
     }
   }
 
-  private static class FlinkFileAppenderFactory implements FileAppenderFactory<RowData> {
+  private static class FlinkFileAppenderFactory implements FileAppenderFactory<RowData>, Serializable {

Review comment:
       Maybe you can let `FileAppenderFactory` extends `Serializable`

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergSinkUtil.java
##########
@@ -37,9 +45,32 @@
 import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
 
 class IcebergSinkUtil {
+
   private IcebergSinkUtil() {
   }
 
+  @SuppressWarnings("unchecked")
+  static DataStreamSink<RowData> write(DataStream<RowData> inputStream,

Review comment:
       If this is a user API, I think a class `FlinkSink` (Instead of IcebergSink?) is more suitable.
   
   And do you think we can let it be a builder kind class?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/RowDataTaskWriterFactory.java
##########
@@ -113,7 +114,7 @@ protected PartitionKey partition(RowData row) {
     }
   }
 
-  private static class FlinkFileAppenderFactory implements FileAppenderFactory<RowData> {
+  private static class FlinkFileAppenderFactory implements FileAppenderFactory<RowData>, Serializable {

Review comment:
       Please add parquet support, and also add tests.

##########
File path: core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java
##########
@@ -27,7 +28,7 @@
 import org.apache.iceberg.encryption.EncryptedOutputFile;
 import org.apache.iceberg.encryption.EncryptionManager;
 
-public class OutputFileFactory {
+public class OutputFileFactory implements Serializable {

Review comment:
       Why?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;

Review comment:
       `maxCommittedCheckpointId` can be a local field?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,

Review comment:
       Why not use `jobId`?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,
+          "There should be an existing iceberg snapshot for current flink job: %s", filesCommitterUid);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(dataFilesPerCheckpoint));

Review comment:
       `checkpointsState.add(dataFilesPerCheckpoint);`

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergSinkUtil.java
##########
@@ -37,9 +45,32 @@
 import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
 
 class IcebergSinkUtil {
+
   private IcebergSinkUtil() {
   }
 
+  @SuppressWarnings("unchecked")
+  static DataStreamSink<RowData> write(DataStream<RowData> inputStream,
+                                       Map<String, String> options,
+                                       Configuration conf,
+                                       String fullTableName,
+                                       Table table,
+                                       TableSchema requestedSchema) {
+    IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, requestedSchema);

Review comment:
       I just notice `IcebergStreamWriter` does not set chaining strategy. Should add in the constructor of `IcebergStreamWriter`: `setChainingStrategy(ChainingStrategy.ALWAYS);`

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergSinkUtil.java
##########
@@ -37,9 +45,32 @@
 import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
 
 class IcebergSinkUtil {
+
   private IcebergSinkUtil() {
   }
 
+  @SuppressWarnings("unchecked")
+  static DataStreamSink<RowData> write(DataStream<RowData> inputStream,
+                                       Map<String, String> options,
+                                       Configuration conf,
+                                       String fullTableName,
+                                       Table table,
+                                       TableSchema requestedSchema) {
+    IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, requestedSchema);
+
+    String filesCommitterUID = String.format("IcebergFilesCommitter-%s", UUID.randomUUID().toString());
+    IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(filesCommitterUID, fullTableName, options, conf);
+
+    DataStream<Void> returnStream = inputStream
+        .transform(IcebergStreamWriter.class.getSimpleName(), TypeInformation.of(DataFile.class), streamWriter)
+        .setParallelism(inputStream.getParallelism())
+        .transform(IcebergFilesCommitter.class.getSimpleName(), Types.VOID, filesCommitter)
+        .setParallelism(1)
+        .setMaxParallelism(1);
+
+    return returnStream.addSink(new DiscardingSink()).setParallelism(1);

Review comment:
       Please name the sink.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";

Review comment:
       `checkpoint-id`?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,

Review comment:
       IIRC, Flink checkpoint id starts at zero. So `maxCommittedCheckpointId >= 0`?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,

Review comment:
       Maybe should be `checkState`.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergSinkUtil.java
##########
@@ -37,9 +45,32 @@
 import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
 
 class IcebergSinkUtil {
+
   private IcebergSinkUtil() {
   }
 
+  @SuppressWarnings("unchecked")
+  static DataStreamSink<RowData> write(DataStream<RowData> inputStream,
+                                       Map<String, String> options,
+                                       Configuration conf,
+                                       String fullTableName,
+                                       Table table,
+                                       TableSchema requestedSchema) {

Review comment:
       Actually, I think it is just `flinkSchema`.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,
+          "There should be an existing iceberg snapshot for current flink job: %s", filesCommitterUid);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(dataFilesPerCheckpoint));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    commitUpToCheckpoint(checkpointId);
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    if (!pendingDataFiles.isEmpty()) {
+      AppendFiles appendFiles = table.newAppend();
+      pendingDataFiles.forEach(appendFiles::appendFile);
+      appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+      appendFiles.set(GLOBAL_FILES_COMMITTER_UID, filesCommitterUid);
+      appendFiles.commit();

Review comment:
       Maybe we should commit even `pendingDataFiles` is empty for `MAX_COMMITTED_CHECKPOINT_ID`.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";

Review comment:
       uid -> job-id?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479449589



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())
+          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)

Review comment:
       committer is a stateful operator, we should probably explicitly set uid. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r470530577



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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 static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  private final String path;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  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);

Review comment:
       That sounds good, I've implemented it in this patch:  https://github.com/apache/iceberg/pull/1185/commits/15dd8e0f3f9c294ece18af9686bd469ea3999e68




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r474420170



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,
+          "There should be an existing iceberg snapshot for current flink job: %s", filesCommitterUid);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(dataFilesPerCheckpoint));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    commitUpToCheckpoint(checkpointId);
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    if (!pendingDataFiles.isEmpty()) {
+      AppendFiles appendFiles = table.newAppend();
+      pendingDataFiles.forEach(appendFiles::appendFile);
+      appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+      appendFiles.set(GLOBAL_FILES_COMMITTER_UID, filesCommitterUid);
+      appendFiles.commit();

Review comment:
       I'm not sure whether the following scenarios are common: however, if there is no data continuously, the previous snapshot may have expired, and an error will be reported after the Flink job failover.
   Continue to write snapshots so that others or monitor can feel: Hey, I'm alive, although I don't write 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r460362001



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergDataStream.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.util.UUID;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+
+public class IcebergDataStream {
+
+  private static final String ICEBERG_STREAM_WRITER = "Iceberg-Stream-Writer";
+  private static final String ICEBERG_FILES_COMMITTER = "Iceberg-Files-Committer";
+
+  private final DataStream<Row> dataStream;
+  private final String path;
+  private final Configuration conf;
+  private final TableSchema tableSchema;
+  private final Integer parallelism;
+
+  private IcebergDataStream(DataStream<Row> dataStream, String path,
+                            Configuration conf, TableSchema tableSchema,
+                            Integer parallelism) {
+    this.dataStream = dataStream;
+    this.path = path;
+    this.conf = conf;
+    this.tableSchema = tableSchema;
+    this.parallelism = parallelism;
+  }
+
+  public static Builder buildFor(DataStream<Row> dataStream) {
+    return new Builder().dataStream(dataStream);
+  }
+
+  public static class Builder {
+    private DataStream<Row> dataStream;
+    private String path;
+    private Configuration conf;
+    private TableSchema tableSchema;
+    private Integer parallelism;
+
+    private Builder dataStream(DataStream<Row> newDataStream) {
+      this.dataStream = newDataStream;
+      return this;
+    }
+
+    public Builder path(String newPath) {
+      this.path = newPath;
+      return this;
+    }
+
+    public Builder config(Configuration newConfig) {
+      this.conf = newConfig;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    public Builder parallelism(int newParallelism) {
+      this.parallelism = newParallelism;
+      return this;
+    }
+
+    public IcebergDataStream build() {
+      return new IcebergDataStream(dataStream, path, conf, tableSchema, parallelism);
+    }
+  }
+
+  public void append() {
+    IcebergStreamWriter streamWriter = IcebergStreamWriter.createStreamWriter(path, tableSchema, conf);
+    IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(path, conf);
+
+    SingleOutputStreamOperator<DataFile> operator = dataStream
+        .transform(ICEBERG_STREAM_WRITER, DataFileTypeInfo.TYPE_INFO, streamWriter)
+        .uid(UUID.randomUUID().toString());
+
+    if (parallelism != null && parallelism > 0) {
+      operator.setParallelism(parallelism);
+    }
+
+    operator.addSink(filesCommitter)
+        .name(ICEBERG_FILES_COMMITTER)
+        .uid(UUID.randomUUID().toString())

Review comment:
       Setting the uid to be a randomly generated number is not a good practice. Operator uid's should be stable, such that between intentional redeployments or simply on restarts due to lost task managers, Flink will know which operators each state belongs to. 
   
   If the uid is changed between restarts, state could be lost.
   
   You can find more information about the use of operator uid's with flink's savepoints (intentional restarts) and why stateful operators should be assigned a stable uid here: https://ci.apache.org/projects/flink/flink-docs-stable/ops/upgrading.html#matching-operator-state




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r480117215



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));

Review comment:
       Considered this issue again,  if redeploying flink job from external checkpoint, then all data files from `restoredDataFiles`  are from the old flink job, which means its checkpoint id could be range from 1~N.  If we put all those files into `dataFilesPerCheckpoint`, that will introduce the problem:  it did not align with the checkpoint id with the new flink job,  saying when `notifyCheckpointComplete(1)` for the new flink job, it won't  commit all the old data files with checkpointId > 1 to iceberg table.  That's incorrect. 
   
   So it should commit those remaining uncommitted data files to iceberg immediately.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478773401



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    tableLoader.open(hadoopConf.get());

Review comment:
       Does this need to be closed?
   
   https://github.com/apache/iceberg/pull/1346/files#r474605890




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r481249647



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));

Review comment:
       @openinx if the new job is redeployed from checkpoint N taken by the old job, checkpointId will start from N+1 for the new job. that is my observation of Flink behavior.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479457104



##########
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:
       commit failure certain isn't common. Agree that it shouldn't be a blocker for the initial version. Just to provide more context on why it is important to us.
   
   Our data warehouses (metastore) only lives in us-east-1 region, while Flink streaming jobs can run in 3 regions (us-east-1, us-west-2, and eu-west-1). 
   
   As Ryan mentioned, we are more concerned about extended outages (like a day) for whatever reason (us-east-1 outage, cross-region network issue, metastore service outage).  This manifest approach allows the Flink jobs to handle those extended outages better.
   
   Flink operator list state can't handle large state well. I vaguely remember 1 or 2 GBs. And it can get pretty slow when the list is large.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479541349



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {

Review comment:
       I am wondering if Flink guarantees the serialized execution for notifyCheckpointComplete?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] tangchenyang commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
tangchenyang commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-688729139


   As a user, how can i use flink write my DataStream to iceberg.
   Is there an example or API document or user document for this feature? 


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r481587012



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));

Review comment:
       OK , checked the flink savepoint code here,  it was designed as you said. https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java#L1371 . 
   
   Thanks for the confirmation,  then I think committing those files in next checkpoint cycle  should also be OK, but there's no harmness for us to commit them immediately.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r460362001



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergDataStream.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.util.UUID;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+
+public class IcebergDataStream {
+
+  private static final String ICEBERG_STREAM_WRITER = "Iceberg-Stream-Writer";
+  private static final String ICEBERG_FILES_COMMITTER = "Iceberg-Files-Committer";
+
+  private final DataStream<Row> dataStream;
+  private final String path;
+  private final Configuration conf;
+  private final TableSchema tableSchema;
+  private final Integer parallelism;
+
+  private IcebergDataStream(DataStream<Row> dataStream, String path,
+                            Configuration conf, TableSchema tableSchema,
+                            Integer parallelism) {
+    this.dataStream = dataStream;
+    this.path = path;
+    this.conf = conf;
+    this.tableSchema = tableSchema;
+    this.parallelism = parallelism;
+  }
+
+  public static Builder buildFor(DataStream<Row> dataStream) {
+    return new Builder().dataStream(dataStream);
+  }
+
+  public static class Builder {
+    private DataStream<Row> dataStream;
+    private String path;
+    private Configuration conf;
+    private TableSchema tableSchema;
+    private Integer parallelism;
+
+    private Builder dataStream(DataStream<Row> newDataStream) {
+      this.dataStream = newDataStream;
+      return this;
+    }
+
+    public Builder path(String newPath) {
+      this.path = newPath;
+      return this;
+    }
+
+    public Builder config(Configuration newConfig) {
+      this.conf = newConfig;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    public Builder parallelism(int newParallelism) {
+      this.parallelism = newParallelism;
+      return this;
+    }
+
+    public IcebergDataStream build() {
+      return new IcebergDataStream(dataStream, path, conf, tableSchema, parallelism);
+    }
+  }
+
+  public void append() {
+    IcebergStreamWriter streamWriter = IcebergStreamWriter.createStreamWriter(path, tableSchema, conf);
+    IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(path, conf);
+
+    SingleOutputStreamOperator<DataFile> operator = dataStream
+        .transform(ICEBERG_STREAM_WRITER, DataFileTypeInfo.TYPE_INFO, streamWriter)
+        .uid(UUID.randomUUID().toString());
+
+    if (parallelism != null && parallelism > 0) {
+      operator.setParallelism(parallelism);
+    }
+
+    operator.addSink(filesCommitter)
+        .name(ICEBERG_FILES_COMMITTER)
+        .uid(UUID.randomUUID().toString())

Review comment:
       Setting the uid to be a randomly generated number is not a good practice. Operator uid's should be stable, such that between intentional redeployments or simply on restarts due to lost task managers, Flink will know which operators each state belongs to. 
   
   If the uid is changed between restarts, state could be lost.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479865410



##########
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:
       The manifest approach sounds a good improvement to me, it's  a todo issue in the next improvement issue. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479589663



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(checkpointId);
+      this.maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    AppendFiles appendFiles = table.newAppend();
+    pendingDataFiles.forEach(appendFiles::appendFile);
+    appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+    appendFiles.set(FLINK_JOB_ID, flinkJobId);
+    appendFiles.commit();
+
+    // Clear the committed data files from dataFilesPerCheckpoint.
+    pendingFileMap.clear();
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFile> element) {
+    this.dataFilesOfCurrentCheckpoint.add(element.getValue());

Review comment:
       this probably gets complicated when we allow concurrent checkpoints. Committer can receive files from both checkpoint N and N+1. We probably need add the checkpointId in the DataFile. It might make sense to provide `FlinkDataFile` wrapper so that we can add Flink additional metadata.
   
   We have `FlinkDataFile` in our implementation for transmitting low and high timestamps. Now thinking about this issue. Maybe we can include checkpointId too so that committer can distinguish data files from different checkpoints.
   ```
   public class FlinkDataFile implements Serializable {
     private final long lowWatermark;
     private final long highWatermark;
     private final DataFile dataFile;
   ```
   
   This does imposes additional requirement on the writer. It needs to know the last/next checkpointId. 
   * for job started without checkpoint, the last checkpointId is 0
   * for job started with checkpoint, now IcebergWriter needs to know the last checkpointId retored.
   
   I couldn't find the checkpointId from the restored context in the `initializeState(context)` method for either `AbstractStreamOperator` or `RichSinkFunction`. It will be nice if it can be exposed. 
   
   Alternatively, we can store the nextCheckpointId in the operator state. However, they also have some problems.
   * operator list state can't deal with rescale as new subtasks won't get state
   * operator union list state is not scalable. Kafka source is suffering the scalability issue with union state.
   
   Note that we may flush file before checkpoint barrier comes. We have two use cases to need the rollover by time and file size.
   * As I mentioned in another comment, we have Flink streaming jobs running in 3 regions in AWS and data warehouses live only in us-east-1. There is a backend service monitor files in two other remote regions and lift/copy them back to the us-east-1 home region. S3 cross-region file copy has 5 GB limit. For that reason, we flushes files if the size reaches 4 GB. 
   * When we implement rough ordering for Iceberg source, we need event time alignment. In these cases, we use Kafka broker time as event time since our Iceberg source tries to emulate the rough ordering for Kafka source. One possible solution is to roll over the file when the min and max timestamp reached certain threshold (e.g. 5 minutes).




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479435913



##########
File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.FiniteTestSource;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFlinkIcebergSink extends AbstractTestBase {
+  private static final Configuration CONF = new Configuration();
+  private static final TypeInformation<Row> ROW_TYPE_INFO = new RowTypeInfo(
+      SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+  private static final DataFormatConverters.RowConverter CONVERTER = new DataFormatConverters.RowConverter(
+      SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes());
+
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  private String tablePath;
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private final FileFormat format;
+  private final int parallelism;
+  private final boolean partitioned;
+
+  @Parameterized.Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] {"avro", 1, true},
+        new Object[] {"avro", 1, false},
+        new Object[] {"avro", 2, true},
+        new Object[] {"avro", 2, false},
+        new Object[] {"orc", 1, true},
+        new Object[] {"orc", 1, false},
+        new Object[] {"orc", 2, true},
+        new Object[] {"orc", 2, false},
+        new Object[] {"parquet", 1, true},
+        new Object[] {"parquet", 1, false},
+        new Object[] {"parquet", 2, true},
+        new Object[] {"parquet", 2, false}
+    };
+  }
+
+  public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) {
+    this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));
+    this.parallelism = parallelism;
+    this.partitioned = partitioned;
+  }
+
+  @Before
+  public void before() throws IOException {
+    File folder = tempFolder.newFolder();
+    String warehouse = folder.getAbsolutePath();
+
+    tablePath = warehouse.concat("/test");
+    Assert.assertTrue("Should create the table path correctly.", new File(tablePath).mkdir());
+
+    Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name());
+    table = SimpleDataUtil.createTable(tablePath, props, partitioned);
+
+    env = StreamExecutionEnvironment.getExecutionEnvironment()
+        .enableCheckpointing(100)
+        .setParallelism(parallelism)
+        .setMaxParallelism(parallelism);
+
+    tableLoader = TableLoader.fromHadoopTable(tablePath);
+  }
+
+  private List<RowData> convertToRowData(List<Row> rows) {
+    return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList());
+  }
+
+  @Test
+  public void testWriteRowData() throws Exception {
+    List<Row> rows = Lists.newArrayList(
+        Row.of(1, "hello"),
+        Row.of(2, "world"),
+        Row.of(3, "foo")
+    );
+    DataStream<RowData> dataStream = env.addSource(new FiniteTestSource<>(rows), ROW_TYPE_INFO)
+        .map(CONVERTER::toInternal, RowDataTypeInfo.of(SimpleDataUtil.ROW_TYPE));
+
+    FlinkSink.forRowData(dataStream)
+        .table(table)
+        .tableLoader(tableLoader)

Review comment:
       Minor: it seems odd to provide the loader and the table. Couldn't the loader be called in the sink builder?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479874623



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);

Review comment:
       It's a great point that we could handle, I think we can attach both the `flinkJobId` and `maxCommittedCheckPointId` to the checkpoint state.  when in restoring path,  we read the `flinkJobId` and `maxCommittedCheckpointId` from states, and compare the current flink job id with `flinkJobId` from state.  If the job ids are matched,  then  it is surely be the case that restoring without redploying(case#1), otherwise it's the case you said (case#2).
   
   For case#1, the current code should be correct. 
   For case#2, we should use the old `flinkJobId` from state to parse the `maxCommittedCheckpointId` in iceberg table, and use that checkpoint id to filter all the committed data files. 
   
   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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479434458



##########
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:
       @stevenzwu ^^




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r472874781



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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 static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  private final String path;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  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, Map<String, String> options, Configuration conf) {
+    this.path = path;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(path, options, conf.get());
+    table = icebergCatalog.loadTable(TableIdentifier.parse(path));
+    maxCommittedCheckpointId = parseMaxCommittedCheckpointId(table.currentSnapshot());
+
+    dataFilesPerCheckpoint = Maps.newTreeMap();
+    dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      dataFilesPerCheckpoint = deserializeState(checkpointsState.get().iterator().next());
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(serializeState(dataFilesPerCheckpoint)));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.tailMap(maxCommittedCheckpointId, false);

Review comment:
       That's a great case (which I did not consider) to proof your point. Yes, we need the `headMap(checkpointId, true)` to avoid future data files being committed to iceberg table,  I've fixed this point in https://github.com/apache/iceberg/pull/1185/commits/598f2353ecad5eda730a1dda96275e552657e466.  




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479863322



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())

Review comment:
       Make sense to me,  could be a following issue. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-673852945


   Let me address above comments and fix the failed unit test.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r473791375



##########
File path: core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java
##########
@@ -27,7 +28,7 @@
 import org.apache.iceberg.encryption.EncryptedOutputFile;
 import org.apache.iceberg.encryption.EncryptionManager;
 
-public class OutputFileFactory {
+public class OutputFileFactory implements Serializable {

Review comment:
       `outputFileFactory` should be `transient` in RowDataTaskWriterFactory, it is inited in `initialize`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478770364



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);
+  }
+
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<Row> rowInput = null;
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder forRow(DataStream<Row> newRowInput) {
+      this.rowInput = newRowInput;
+      return this;
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    private DataStream<RowData> convert() {
+      Preconditions.checkArgument(rowInput != null, "The DataStream<Row> to convert shouldn't be null");

Review comment:
       The preconditions in build ensure this is never the case. I'm fine keeping the check (up to you) but it seems odd to have it in a private method. Also, this relies on the check that `table` is not null when `tableSchema` is null.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479451331



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())

Review comment:
       this is a good default value for writer parallelism. we have users who want to explicitly control the writer parallelism to control the number of written files. in the future, we may want to allow user to set parallelism in the builder.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-683164685


   nit: on class names
   
   IcebergStreamWriter -> IcebergWriter, because it also BoundedOneInput interface, which seems like a batch job.
   
   IcebergFilesCommitter -> IcebergCommitter. Iceberg only commits files. So including Files seems redundant.
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r475369748



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.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.ChainingStrategy;
+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.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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+    setChainingStrategy(ChainingStrategy.ALWAYS);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    tableLoader.open(hadoopConf.get());
+    table = tableLoader.loadTable();
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(checkpointId);
+      maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    AppendFiles appendFiles = table.newAppend();
+    pendingDataFiles.forEach(appendFiles::appendFile);
+    appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+    appendFiles.set(FLINK_JOB_ID, flinkJobId);
+    appendFiles.commit();
+
+    // Clear the committed data files from dataFilesPerCheckpoint.
+    pendingFileMap.clear();
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFile> element) {
+    this.dataFilesOfCurrentCheckpoint.add(element.getValue());
+  }
+
+  @Override
+  public void endInput() {
+    commitUpToCheckpoint(Long.MAX_VALUE);
+  }
+
+  private static ListStateDescriptor<SortedMap<Long, List<DataFile>>> buildStateDescriptor() {
+    Comparator<Long> longComparator = Comparators.forType(Types.LongType.get());
+    // Construct a ListTypeInfo.
+    ListTypeInfo<DataFile> dataFileListTypeInfo = new ListTypeInfo<>(TypeInformation.of(DataFile.class));
+    // Construct a SortedMapTypeInfo.
+    SortedMapTypeInfo<Long, List<DataFile>> sortedMapTypeInfo = new SortedMapTypeInfo<>(
+        BasicTypeInfo.LONG_TYPE_INFO, dataFileListTypeInfo, longComparator
+    );
+    return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo);
+  }
+
+  static Long getMaxCommittedCheckpointId(Table table, String flinkJobId) {

Review comment:
       For me, initializing the `maxCommittedCheckpointId`  to  -1 seems simpler to handle, because when calling `SortedMap#tailMap` or `SortedMap#headMap` we don't need to check nullable. The comparasion in `notifyCheckpointComplete` is the similar point.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479527941



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);

Review comment:
       @openinx I would prefer to avoid the extra conversion. E.g. we use Avro for streaming data and the `ParquetAvroWriter` from Iceberg library for writer function. On the other hand, we can supply an Avro TaskWriterFactory to `IcebergStreamWriter`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r480208355



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())
+          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)
+          .setParallelism(1)
+          .setMaxParallelism(1);
+
+      return returnStream.addSink(new DiscardingSink())

Review comment:
       @openinx thx. that does answer my question. 
   
   Still adding a DiscardingSink may confuse users. It seems that we really need a unified/improved sink interface (similar to FLIP-27) to supported bounded input jobs in sink function.
   
   What about any other Flink sinks used by bounded streaming/batch job? Do they all have to go through this model? 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r469812741



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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 static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  private final String path;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  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, Map<String, String> options, Configuration conf) {
+    this.path = path;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(path, options, conf.get());
+    table = icebergCatalog.loadTable(TableIdentifier.parse(path));
+    maxCommittedCheckpointId = parseMaxCommittedCheckpointId(table.currentSnapshot());
+
+    dataFilesPerCheckpoint = Maps.newTreeMap();
+    dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      dataFilesPerCheckpoint = deserializeState(checkpointsState.get().iterator().next());
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(serializeState(dataFilesPerCheckpoint)));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.tailMap(maxCommittedCheckpointId, false);

Review comment:
       Why exclude `maxCommittedCheckpointId` and `checkpointId`? Just `dataFilesPerCheckpoint.tailMap(checkpointId, true)`? I don't understand why we need store previous checkpoint id.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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 static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  private final String path;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  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);

Review comment:
       Can we use Flink `MapTypeInfo`, and addAll to sorted map?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r473792184



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,

Review comment:
       Yes, but I think `maxCommittedCheckpointId != INIT_ID` should be more reasonable.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479450416



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())
+          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)
+          .setParallelism(1)
+          .setMaxParallelism(1);
+
+      return returnStream.addSink(new DiscardingSink())

Review comment:
       curious about the reason why don't we make the committer a sink function and instead add a dummy DiscardingSink.
   
   Conceptually, this writer-committer combo is the reverse/mirror of split enumerator-reader FLIP-27 source interface. It will be nice to run committer on jobmanager (similar to enumerator). This way, Iceberg sink won't change the nature of the embarrassingly-parallel DAG.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479432948



##########
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:
       I completely agree that we should not block adding the sink until this is done. I just wanted to note that this is a good place to put some extra effort. I think that multi-region deployments are common and this can really help. We don't keep Kafka data for a long time, so it is important to us that Flink continues to consume Kafka data to eventually commit to the table, even if it can't commit the manifests right away.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-682956273


   Merged. Thanks @openinx, it is great to have this in. And thanks for helping review, @JingsongLi and @kbendick!


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478769632



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);

Review comment:
       Curious: are there any plans to support other object models? I know the original sink used Avro, which is supported by Avro and Parquet file formats.
   
   Maybe for that we need a better way to configure file formats with object models, though.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479433769



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);

Review comment:
       That sounds okay to me. We can always extend this with a path for other record formats in our version. And I do think it would be much easier to do once we have a more pluggable method to specify object models and data formats.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r470528123



##########
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:
       Added a unit test to address this thing. https://github.com/apache/iceberg/pull/1185/files#diff-67c60f8d1a96e4583f5b53248df15bedR257




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-688741887


   Hi @tangchenyang   The unit test [TestFlinkIcebergSink](https://github.com/apache/iceberg/blob/master/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java#L133)  provide a complete demo to show you how to write records into iceberg table by flink.  I'm writing a document to show users how to write data into iceberg by both table sql and datastream api, but not finished yet. 
   https://github.com/apache/iceberg/pull/1423


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r481275502



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())
+          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)
+          .setParallelism(1)
+          .setMaxParallelism(1);
+
+      return returnStream.addSink(new DiscardingSink())

Review comment:
       fair enough. we can't solve this problem until Flink improved sink interface.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] YesOrNo828 commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
YesOrNo828 commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r464830310



##########
File path: flink/src/main/java/org/apache/iceberg/flink/PartitionKey.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.transforms.Transform;
+import org.apache.iceberg.types.Types;
+
+public class PartitionKey implements StructLike {
+
+  private final Object[] partitionTuple;
+
+  private PartitionKey(Object[] partitionTuple) {
+    this.partitionTuple = partitionTuple;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof PartitionKey)) {
+      return false;
+    }
+
+    PartitionKey that = (PartitionKey) o;
+    return Arrays.equals(partitionTuple, that.partitionTuple);
+  }
+
+  @Override
+  public int hashCode() {
+    return Arrays.hashCode(partitionTuple);
+  }
+
+  @Override
+  public int size() {
+    return partitionTuple.length;
+  }
+
+  @Override
+  public <T> T get(int pos, Class<T> javaClass) {
+    return javaClass.cast(partitionTuple[pos]);
+  }
+
+  public Object[] getPartitionTuple() {
+    return partitionTuple;
+  }
+
+  @Override
+  public <T> void set(int pos, T value) {
+    partitionTuple[pos] = value;
+  }
+
+  private static Map<Integer, Integer> buildFieldId2PosMap(Schema schema) {
+    Map<Integer, Integer> fieldId2Position = Maps.newHashMap();
+    List<Types.NestedField> nestedFields = schema.asStruct().fields();
+    for (int i = 0; i < nestedFields.size(); i++) {
+      fieldId2Position.put(nestedFields.get(i).fieldId(), i);
+    }
+    return fieldId2Position;
+  }
+
+  public static Builder builder(PartitionSpec spec) {
+    return new Builder(spec);
+  }
+
+  public static class Builder {
+    private final int size;
+
+    private final int[] pos;
+    private final Transform[] transforms;
+
+    private Builder(PartitionSpec spec) {
+      List<PartitionField> fields = spec.fields();
+      this.size = fields.size();
+      this.pos = new int[size];
+      this.transforms = new Transform[size];
+
+      Map<Integer, Integer> fieldId2Pos = buildFieldId2PosMap(spec.schema());
+
+      for (int i = 0; i < size; i += 1) {
+        PartitionField field = fields.get(i);
+        Integer position = fieldId2Pos.get(field.sourceId());
+        Preconditions.checkArgument(position != null,
+            "Field source id from PartitionSpec MUST exist in the original schema");
+        this.pos[i] = position;
+        this.transforms[i] = field.transform();
+      }
+    }
+
+    @SuppressWarnings("unchecked")
+    public PartitionKey build(Row row) {
+      Object[] partitionTuple = new Object[size];
+      for (int i = 0; i < partitionTuple.length; i += 1) {
+        partitionTuple[i] = transforms[i].apply(row.getField(pos[i]));

Review comment:
       `pos[i]` It seemed the fields of flink ddl must be the same sequence with the fields of iceberg schema . what if flink ddl misses some fields? I think partition keys' indices will be different from in iceberg schema




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-682960033


   What are the next steps for Flink writes? I think we can probably start writing some documentation. The tests for the sink look like this is mostly complete. Maybe docs are an area that @kbendick could help with?


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r473892173



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,
+          "There should be an existing iceberg snapshot for current flink job: %s", filesCommitterUid);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(dataFilesPerCheckpoint));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    commitUpToCheckpoint(checkpointId);
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    if (!pendingDataFiles.isEmpty()) {
+      AppendFiles appendFiles = table.newAppend();
+      pendingDataFiles.forEach(appendFiles::appendFile);
+      appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+      appendFiles.set(GLOBAL_FILES_COMMITTER_UID, filesCommitterUid);
+      appendFiles.commit();

Review comment:
       The `MAX_COMMITTED_CHECKPOINT_ID`  stored in iceberg table is mainly used for avoiding duplicated data files to be committed to iceberg table.  Now we're sure that there's no data files to commit,  advancing the max-committed-checkpoint-id don't have much value in my thought. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r473815718



##########
File path: core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java
##########
@@ -27,7 +28,7 @@
 import org.apache.iceberg.encryption.EncryptedOutputFile;
 import org.apache.iceberg.encryption.EncryptionManager;
 
-public class OutputFileFactory {
+public class OutputFileFactory implements Serializable {

Review comment:
       OK, it 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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r475329407



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSink.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.util.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final TypeInformation<DataFile> DATA_FILE_TYPE_INFO = TypeInformation.of(DataFile.class);

Review comment:
       NIT:  I think these fields not have to be static.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.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.ChainingStrategy;
+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.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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+    setChainingStrategy(ChainingStrategy.ALWAYS);

Review comment:
       We can let committer be a single node.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.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.ChainingStrategy;
+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.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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+    setChainingStrategy(ChainingStrategy.ALWAYS);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    tableLoader.open(hadoopConf.get());
+    table = tableLoader.loadTable();
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(checkpointId);
+      maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    AppendFiles appendFiles = table.newAppend();
+    pendingDataFiles.forEach(appendFiles::appendFile);
+    appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+    appendFiles.set(FLINK_JOB_ID, flinkJobId);
+    appendFiles.commit();
+
+    // Clear the committed data files from dataFilesPerCheckpoint.
+    pendingFileMap.clear();
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFile> element) {
+    this.dataFilesOfCurrentCheckpoint.add(element.getValue());
+  }
+
+  @Override
+  public void endInput() {
+    commitUpToCheckpoint(Long.MAX_VALUE);
+  }
+
+  private static ListStateDescriptor<SortedMap<Long, List<DataFile>>> buildStateDescriptor() {
+    Comparator<Long> longComparator = Comparators.forType(Types.LongType.get());
+    // Construct a ListTypeInfo.
+    ListTypeInfo<DataFile> dataFileListTypeInfo = new ListTypeInfo<>(TypeInformation.of(DataFile.class));
+    // Construct a SortedMapTypeInfo.
+    SortedMapTypeInfo<Long, List<DataFile>> sortedMapTypeInfo = new SortedMapTypeInfo<>(
+        BasicTypeInfo.LONG_TYPE_INFO, dataFileListTypeInfo, longComparator
+    );
+    return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo);
+  }
+
+  static Long getMaxCommittedCheckpointId(Table table, String flinkJobId) {

Review comment:
       Actually, the default value of `maxCommittedCheckpointId` can be null instead of `INITIAL_CHECKPOINT_ID`.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.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.ChainingStrategy;
+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.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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+    setChainingStrategy(ChainingStrategy.ALWAYS);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    tableLoader.open(hadoopConf.get());

Review comment:
       Better to close this `tableLoader` in the `dispose`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-677775120


   I think this one is ready to review and commit, right?


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r472895281



##########
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:
       That means someone have expired the latest committed snapshot for a running flink job, usually we flink checkpoint interval will be several minutes, while the snapshot expire interval will be several days or weeks.  It's unlikely that we will expire the latest committed snapshot unless someone set the unreasonable intervals. 
   
   On the other hand,  even if someone removed the latest committed snapshot,  the flink job will write the data to iceberg table correctly unless we restore the flink job once snapshot was removed. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-679455842


   Since both parquet reader and writer have been merged, so let me update this patch to add parquet into unit tests. 


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-677991898


   > I think this one is ready to review and commit, right?
   
   That's right. ( After this DataStream sink,  the next patch is https://github.com/apache/iceberg/pull/1348).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478767253



##########
File path: core/src/main/java/org/apache/iceberg/io/FileAppenderFactory.java
##########
@@ -19,14 +19,15 @@
 
 package org.apache.iceberg.io;
 
+import java.io.Serializable;
 import org.apache.iceberg.FileFormat;
 
 /**
  * Factory to create a new {@link FileAppender} to write records.
  *
  * @param <T> data type of the rows to append.
  */
-public interface FileAppenderFactory<T> {
+public interface FileAppenderFactory<T> extends Serializable {

Review comment:
       This isn't required for Spark. Could we add this to the `FlinkFileAppenderFactory` instead?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r472776602



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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 static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  private final String path;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  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, Map<String, String> options, Configuration conf) {
+    this.path = path;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(path, options, conf.get());
+    table = icebergCatalog.loadTable(TableIdentifier.parse(path));
+    maxCommittedCheckpointId = parseMaxCommittedCheckpointId(table.currentSnapshot());
+
+    dataFilesPerCheckpoint = Maps.newTreeMap();
+    dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      dataFilesPerCheckpoint = deserializeState(checkpointsState.get().iterator().next());
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(serializeState(dataFilesPerCheckpoint)));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.tailMap(maxCommittedCheckpointId, false);

Review comment:
       Sorry, wrong reading. Should be`headMap` instead of `tailMap`. You can think again about my above example.
   
   I got your point about `tailMap`. But it is incorrect too. Consider the case:
   - 1.snapshot(cpId)
   - 2.snapshot(cpId + 1)
   - 3.cpId success, notifyComplete(cpId)
   - 4.cpId + 1 success, notifyComplete(cpId + 1)
   
   If there is a failure between 3 and 4, the data in table will be incorrect.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-678969377


   Ping @rdblue , we ( JingsongLi and I ) have reached a basic agreement about this patch.  Would you mind to take a look when you have time ? Thanks.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479585888



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));

Review comment:
       In our implementation, we immediately commit any uncommitted files upon restore to avoid waiting for another checkpoint cycle. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jrthe42 commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
jrthe42 commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r476299202



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSink.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.util.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);
+  }
+
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<Row> rowInput = null;
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder forRow(DataStream<Row> newRowInput) {
+      this.rowInput = newRowInput;
+      return this;
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    private DataStream<RowData> convert() {
+      Preconditions.checkArgument(rowInput != null, "The DataStream<Row> to convert shouldn't be null");
+
+      RowType rowType;
+      DataType[] fieldDataTypes;
+      if (tableSchema != null) {
+        rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+        fieldDataTypes = tableSchema.getFieldDataTypes();
+      } else {
+        rowType = FlinkSchemaUtil.convert(table.schema());
+        fieldDataTypes = TypeConversions.fromLogicalToDataType(rowType.getChildren().toArray(new LogicalType[0]));
+      }
+
+      DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+
+      return rowInput.map(rowConverter::toInternal, RowDataTypeInfo.of(rowType));
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowInput != null || rowDataInput != null,
+          "Should initialize input DataStream first with DataStream<Row> or DataStream<RowData>");
+      Preconditions.checkArgument(rowInput == null || rowDataInput == null,
+          "Could only initialize input DataStream with either DataStream<Row> or DataStream<RowData>");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");

Review comment:
       Since we already have TableLoader, will it be easier for user if we load table here ?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-670607735


   @openinx, can you rebase this now that #1145 has been merged?


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r460362464



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergDataStream.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.util.UUID;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+
+public class IcebergDataStream {
+
+  private static final String ICEBERG_STREAM_WRITER = "Iceberg-Stream-Writer";
+  private static final String ICEBERG_FILES_COMMITTER = "Iceberg-Files-Committer";
+
+  private final DataStream<Row> dataStream;
+  private final String path;
+  private final Configuration conf;
+  private final TableSchema tableSchema;
+  private final Integer parallelism;
+
+  private IcebergDataStream(DataStream<Row> dataStream, String path,
+                            Configuration conf, TableSchema tableSchema,
+                            Integer parallelism) {
+    this.dataStream = dataStream;
+    this.path = path;
+    this.conf = conf;
+    this.tableSchema = tableSchema;
+    this.parallelism = parallelism;
+  }
+
+  public static Builder buildFor(DataStream<Row> dataStream) {
+    return new Builder().dataStream(dataStream);
+  }
+
+  public static class Builder {
+    private DataStream<Row> dataStream;
+    private String path;
+    private Configuration conf;
+    private TableSchema tableSchema;
+    private Integer parallelism;
+
+    private Builder dataStream(DataStream<Row> newDataStream) {
+      this.dataStream = newDataStream;
+      return this;
+    }
+
+    public Builder path(String newPath) {
+      this.path = newPath;
+      return this;
+    }
+
+    public Builder config(Configuration newConfig) {
+      this.conf = newConfig;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    public Builder parallelism(int newParallelism) {
+      this.parallelism = newParallelism;
+      return this;
+    }
+
+    public IcebergDataStream build() {
+      return new IcebergDataStream(dataStream, path, conf, tableSchema, parallelism);
+    }
+  }
+
+  public void append() {
+    IcebergStreamWriter streamWriter = IcebergStreamWriter.createStreamWriter(path, tableSchema, conf);
+    IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(path, conf);
+
+    SingleOutputStreamOperator<DataFile> operator = dataStream
+        .transform(ICEBERG_STREAM_WRITER, DataFileTypeInfo.TYPE_INFO, streamWriter)
+        .uid(UUID.randomUUID().toString());
+
+    if (parallelism != null && parallelism > 0) {
+      operator.setParallelism(parallelism);
+    }
+
+    operator.addSink(filesCommitter)
+        .name(ICEBERG_FILES_COMMITTER)
+        .uid(UUID.randomUUID().toString())

Review comment:
       Most of the time, I simply assign the `name` as the `uid`. In Scala, I use an implicit class to assign name and uid at the same time with the same value. This assumes that the name of an operator is unique (which might not always be true, and is something that should be considered as a library author), but so far I've not run into issues with the handful of jobs that I oversee.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478770364



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);
+  }
+
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<Row> rowInput = null;
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder forRow(DataStream<Row> newRowInput) {
+      this.rowInput = newRowInput;
+      return this;
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    private DataStream<RowData> convert() {
+      Preconditions.checkArgument(rowInput != null, "The DataStream<Row> to convert shouldn't be null");

Review comment:
       The preconditions in build ensure this is never the case. I'm fine keeping the check (up to you) but it seems odd to have it in a private method.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] tangchenyang commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
tangchenyang commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-688747036


   Greate!  Many thanks. @openinx 


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479589663



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(checkpointId);
+      this.maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    AppendFiles appendFiles = table.newAppend();
+    pendingDataFiles.forEach(appendFiles::appendFile);
+    appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+    appendFiles.set(FLINK_JOB_ID, flinkJobId);
+    appendFiles.commit();
+
+    // Clear the committed data files from dataFilesPerCheckpoint.
+    pendingFileMap.clear();
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFile> element) {
+    this.dataFilesOfCurrentCheckpoint.add(element.getValue());

Review comment:
       this probably gets complicated when we allow concurrent checkpoints. Committer can receive files from both checkpoint N and N+1. We probably need add the checkpointId in the DataFile. I think it might make sense to provide `FlinkDataFile` wrapper so that we can add Flink additional metadata.
   
   We have that in our implementation for low and high timestamps. Now thinking about this issue. Maybe we can include checkpointId too so that committer can distinguish data files from different checkpoints.
   ```
   public class FlinkDataFile implements Serializable {
     private final long lowWatermark;
     private final long highWatermark;
     private final DataFile dataFile;
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479457104



##########
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:
       Just to provide more context on why we need it. Our data warehouses (metastore) only lives in us-east-1 region, while Flink streaming jobs can run in 3 regions (us-east-1, us-west-2, and eu-west-1). Transient commits failures happen from time to time, which aren't big concerns. We are more concerned about extended outages (like a day) for whatever reason (us-east-1 outage, cross-region network issue, metastore service outage).  For high-parallelism or event time partitioned tables, there could be thousands or tens of thousands of files per checkpoint interval. This manifest approach allows the Flink jobs to handle those extended outages better.
   
   Flink operator list state can't handle large state well. I vaguely remember limit is 1 or 2 GBs. And it can get pretty slow when the list is large.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479876458



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {

Review comment:
       According to the current implementation, it should be, because task checkpoints are ordered and RPC messages are ordered. But it's better not to rely on this.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r477186882



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSink.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.util.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);
+  }
+
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<Row> rowInput = null;
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder forRow(DataStream<Row> newRowInput) {
+      this.rowInput = newRowInput;
+      return this;
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    private DataStream<RowData> convert() {
+      Preconditions.checkArgument(rowInput != null, "The DataStream<Row> to convert shouldn't be null");
+
+      RowType rowType;
+      DataType[] fieldDataTypes;
+      if (tableSchema != null) {
+        rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+        fieldDataTypes = tableSchema.getFieldDataTypes();
+      } else {
+        rowType = FlinkSchemaUtil.convert(table.schema());
+        fieldDataTypes = TypeConversions.fromLogicalToDataType(rowType.getChildren().toArray(new LogicalType[0]));
+      }
+
+      DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+
+      return rowInput.map(rowConverter::toInternal, RowDataTypeInfo.of(rowType));
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowInput != null || rowDataInput != null,
+          "Should initialize input DataStream first with DataStream<Row> or DataStream<RowData>");
+      Preconditions.checkArgument(rowInput == null || rowDataInput == null,
+          "Could only initialize input DataStream with either DataStream<Row> or DataStream<RowData>");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");

Review comment:
       The `table` is used to initialize `IcebergStreamWriter`  at client side,  while the `tableLoader` is used to load table at `TaskManager` side,   using the `tableLoader` for `TaskManager` to load table for  client side looks strange to me. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r480625043



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())
+          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)
+          .setParallelism(1)
+          .setMaxParallelism(1);
+
+      return returnStream.addSink(new DiscardingSink())

Review comment:
       There're three cases: 1>  unbounded streaming job; 2> bounded streaming job;  3> batch job.  If users only need the `unbounded streaming` ability, then only need to implement the `SinkFunction`,  otherwise if need both `unbounded streaming` and `bounded streaming`  ability,  then we need to extend/implement the `AbstractStreamOperator` & `BoundedOneInput`  and add the `DiscardingSink` to the tail.  If want to `batch` ability, then  need to provide an `OutputFormat` implementation.  In future flink,  we will unify the case#2 and case#3 in one sink interface,  but for now we have to implement separately for the bounded streaming and batch cases. 
   
   Flink hive connector is a good case, which have support case1, case2 and case3.  It also use the similar way to the current iceberg sink connector now. 
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479875612



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(checkpointId);
+      this.maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {

Review comment:
       That should be similar to this point from this [comment](https://github.com/apache/iceberg/pull/1185#discussion_r474420170), we actually are doing the logic you said and provided a unit test to address this thing. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479589663



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(checkpointId);
+      this.maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    AppendFiles appendFiles = table.newAppend();
+    pendingDataFiles.forEach(appendFiles::appendFile);
+    appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+    appendFiles.set(FLINK_JOB_ID, flinkJobId);
+    appendFiles.commit();
+
+    // Clear the committed data files from dataFilesPerCheckpoint.
+    pendingFileMap.clear();
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFile> element) {
+    this.dataFilesOfCurrentCheckpoint.add(element.getValue());

Review comment:
       this probably gets complicated when we allow concurrent checkpoints. Committer can receive files from both checkpoint N and N+1. We probably need add the checkpointId in the DataFile. It might make sense to provide `FlinkDataFile` wrapper so that we can add Flink additional metadata.
   
   We have `FlinkDataFile` in our implementation for transmitting low and high timestamps. Now thinking about this issue. Maybe we can include checkpointId too so that committer can distinguish data files from different checkpoints.
   ```
   public class FlinkDataFile implements Serializable {
     private final long lowWatermark;
     private final long highWatermark;
     private final DataFile dataFile;
   ```
   
   This does imposes additional requirement on the writer. It needs to know the next checkpointId. I couldn't find the checkpointId from the context of the `initializeState(context)` method for either `AbstractStreamOperator` or `RichSinkFunction`. It will be nice if it can be exposed. 
   
   Alternatively, we can store the nextCheckpointId in the operator state.
   * operator list state can't deal with rescale as new subtasks won't get state
   * operator union list state is not scalable. Kafka source is suffering the scalability issue with union state.
   
   Note that we may flush file before checkpoint barrier comes. E.g. As I mentioned in another comment, we have Flink streaming jobs running in 3 regions in AWS and data warehouses live only in us-east-1. There is a backend service monitor files in two other remote regions and lift/copy them back to the us-east-1 home region. S3 cross-region file copy has 5 GB limit. For that reason, we flushes files if the size reaches 4 GB




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-678073729


   Hi @openinx , for the API level, I think we can do more:
   - We can provide two API, one for RowData, one for Row, This can cover not only most users of `DataStream`, but also the requirements of SQL layer.
   - Users can just construct a sink from `DataStream` and `TableLoader`, other information can be inferred.
   
   Just like:
   ```
   public class FlinkSink {
     private FlinkSink() {
     }
   
     public static Builder<Row> forRow(DataStream<Row> input) {
       return new Builder<>(input);
     }
   
     public static Builder<RowData> forRowData(DataStream<RowData> input) {
       return new Builder<>(input);
     }
   
     private static class Builder<T> {
       private final DataStream<T> input;
       private TableLoader loader;
       private Configuration hadoopConf;
       private Table table;
       private TableSchema tableSchema;
   
       // ---------- Required options ------------
   
       private Builder(DataStream<T> input) {
         this.input = input;
       }
   
       public Builder tableLoader(TableLoader loader) {
         this.loader = loader;
         return this;
       }
   
       // ---------- Optional options ------------
   
       public Builder table(Table newTable) {
         this.table = newTable;
         return this;
       }
   
       public Builder hadoopConf(Configuration newConf) {
         this.hadoopConf = newConf;
         return this;
       }
   
       public Builder tableSchema(TableSchema newSchema) {
         this.tableSchema = newSchema;
         return this;
       }
   
       @SuppressWarnings("unchecked")
       public DataStreamSink<RowData> build() {
         Preconditions.checkNotNull(input, "Input data stream shouldn't be null");
         Preconditions.checkNotNull(loader, "Table loader shouldn't be null");
   
         if (hadoopConf == null) {
           // load cluster conf
         }
   
         if (table == null) {
           // load table from table loader
         }
   
         // tableSchema can be optional
   
         DataStream<RowData> inputStream;
         Class<T> inputClass = input.getType().getTypeClass();
         if (inputClass == Row.class) {
           DataType type;
           if (tableSchema != null) {
             type = tableSchema.toRowDataType();
           } else {
             type = TypeConversions.fromLogicalToDataType(FlinkSchemaUtil.convert(table.schema()));
           }
           DataStructureConverter converter = DataStructureConverters.getConverter(type);
           inputStream = input.map((MapFunction) converter::toInternal);
         } else if (inputClass == RowData.class) {
           inputStream = (DataStream<RowData>) input;
         } else {
           throw new IllegalArgumentException("Should be Row or RowData");
         }
   
         // create writer form inputStream.
         // create committer.
         // return DataStreamSink.
         return null;
       }
     }
   }
   ```


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r474461224



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,
+          "There should be an existing iceberg snapshot for current flink job: %s", filesCommitterUid);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(dataFilesPerCheckpoint));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    commitUpToCheckpoint(checkpointId);
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    if (!pendingDataFiles.isEmpty()) {
+      AppendFiles appendFiles = table.newAppend();
+      pendingDataFiles.forEach(appendFiles::appendFile);
+      appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+      appendFiles.set(GLOBAL_FILES_COMMITTER_UID, filesCommitterUid);
+      appendFiles.commit();

Review comment:
       The case you described should be a rare case,  but I agree with you that we'd better to update the max-committed-checkpoint-id, so that the failover won't fail.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478770155



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);
+  }
+
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<Row> rowInput = null;
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder forRow(DataStream<Row> newRowInput) {
+      this.rowInput = newRowInput;
+      return this;
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    private DataStream<RowData> convert() {
+      Preconditions.checkArgument(rowInput != null, "The DataStream<Row> to convert shouldn't be null");
+
+      RowType rowType;
+      DataType[] fieldDataTypes;
+      if (tableSchema != null) {
+        rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+        fieldDataTypes = tableSchema.getFieldDataTypes();
+      } else {
+        rowType = FlinkSchemaUtil.convert(table.schema());
+        fieldDataTypes = TypeConversions.fromLogicalToDataType(rowType.getChildren().toArray(new LogicalType[0]));
+      }
+
+      DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+
+      return rowInput.map(rowConverter::toInternal, RowDataTypeInfo.of(rowType));
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowInput != null || rowDataInput != null,
+          "Should initialize input DataStream first with DataStream<Row> or DataStream<RowData>");

Review comment:
       I don't think these messages are very helpful. The problem is that neither `forRow` or `forRowData` is called. Being specific about how to fix it (call one of those methods) would be a more helpful error.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479450416



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())
+          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)
+          .setParallelism(1)
+          .setMaxParallelism(1);
+
+      return returnStream.addSink(new DiscardingSink())

Review comment:
       curious about the reason why don't we make the committer a sink function and instead add a dummy DiscardingSink.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r473771348



##########
File path: core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java
##########
@@ -27,7 +28,7 @@
 import org.apache.iceberg.encryption.EncryptedOutputFile;
 import org.apache.iceberg.encryption.EncryptionManager;
 
-public class OutputFileFactory {
+public class OutputFileFactory implements Serializable {

Review comment:
       Because the `IcebergStreamWriter` will need the `RowDataTaskWriterFactory` to create `TaskWriter`,  it could pass it to `TaskWriterFactory` when constructing, that means all members inside `RowDataTaskWriterFactory` should be serializable (include itself), unfortunately `OutputFileFactory` is one of its member. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-673357802


   @openinx and @rdblue I think we can consider https://github.com/apache/iceberg/pull/1332 , what do you think?


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r473758453



##########
File path: flink/src/main/java/org/apache/iceberg/flink/RowDataTaskWriterFactory.java
##########
@@ -113,7 +114,7 @@ protected PartitionKey partition(RowData row) {
     }
   }
 
-  private static class FlinkFileAppenderFactory implements FileAppenderFactory<RowData> {
+  private static class FlinkFileAppenderFactory implements FileAppenderFactory<RowData>, Serializable {

Review comment:
       We need to wait until the parquet writer is available. https://github.com/apache/iceberg/pull/1272.  Now only the parquet readers get merged. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479588663



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(checkpointId);
+      this.maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {

Review comment:
       Just want to point out that this will commit if there is zero pending files. For us, we actually still want to commit in this case mainly to update the region watermark info.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479448928



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)

Review comment:
       there can be multiple Iceberg sinks in the same job. we probably should add the table identifier string suffix to make operator name and id unique.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479865410



##########
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:
       The manifest approach sounds a good improvement to me, it's  a todo issue in the next improvement https://github.com/apache/iceberg/issues/1403. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479448928



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)

Review comment:
       there can be multiple Iceberg sinks in the same job. we probably should add the table identifier string suffix to make operator name and id unique. We have a unique `sinkName` within a job and we add the `sinkName` suffix for operator name.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r460361671



##########
File path: build.gradle
##########
@@ -253,6 +255,9 @@ project(':iceberg-flink') {
     compileOnly("org.apache.hadoop:hadoop-minicluster") {
       exclude group: 'org.apache.avro', module: 'avro'
     }
+    compileOnly("de.javakaffee:kryo-serializers:0.45") {
+      exclude group: 'com.esotericsoftware', module: 'kryo'
+    }
 
     testCompile "org.apache.flink:flink-core"
     testCompile "org.apache.flink:flink-runtime_2.12"

Review comment:
       Relatively nooby question unrelated to this PR, but does Iceberg only support Flink projects running on Scala 2.12? And is this the same for Spark? For Spark, I would imagine so as Spark 3.0 drops support for Scala 2.11 entirely.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479549779



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())
+          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)

Review comment:
       never mind. saw earlier comment that this is a todo item.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479457104



##########
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:
       commit failure certain isn't common. Agree that it shouldn't be a blocker for the initial version. Just to provide more context on why it is important to us.
   
   Our data warehouses (metastore) only lives in us-east-1 region, while Flink streaming jobs can run in 3 regions (us-east-1, us-west-2, and eu-west-1). 
   
   As Ryan mentioned, we are more concerned about extended outages (like a day) for whatever reason (us-east-1 outage, cross-region network issue, metastore service outage).  For high-parallelism or event time partitioned tables, there could be thousands or tens of thousands of files per checkpoint interval. This manifest approach allows the Flink jobs to handle those extended outages better.
   
   Flink operator list state can't handle large state well. I vaguely remember 1 or 2 GBs. And it can get pretty slow when the list is large.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479545979



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);

Review comment:
       We might have a problem if redeploying the Flink job from external checkpoint. It is a new `flinkJobId` in this case. maxCommittedCheckpointId will be `-1`.  As a result, we can commit those committed files again later.
   
   The way we do de-dup is to generate a hash for the manifest file path and store the hash in snapshot summary. During restore, we use the hash to de-dup if the manifest file was committed or not.
   
   ```
           List<String> hashes = new ArrayList<>(flinkManifestFiles.size());
           AppendFiles appendFiles = transaction.newAppend();
           for (FlinkManifestFile flinkManifestFile : flinkManifestFiles) {
             appendFiles.appendManifest(flinkManifestFile);
             hashes.add(flinkManifestFile.hash());
           }
           appendFiles.set(
               COMMIT_MANIFEST_HASHES_KEY, FlinkManifestFileUtil.hashesListToString(hashes));
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r473733294



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,

Review comment:
       Maybe just `maxCommittedCheckpointId != INIT_ID`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r472203477



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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 static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  private final String path;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  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, Map<String, String> options, Configuration conf) {
+    this.path = path;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(path, options, conf.get());
+    table = icebergCatalog.loadTable(TableIdentifier.parse(path));
+    maxCommittedCheckpointId = parseMaxCommittedCheckpointId(table.currentSnapshot());
+
+    dataFilesPerCheckpoint = Maps.newTreeMap();
+    dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      dataFilesPerCheckpoint = deserializeState(checkpointsState.get().iterator().next());
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(serializeState(dataFilesPerCheckpoint)));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.tailMap(maxCommittedCheckpointId, false);

Review comment:
       > the data of the current checkpoint needs to be committed in the next cycle.
   
   That's not correct. Since we've merged the `dataFilesOfCurrentCheckpoint` into `dataFilesPerCheckpoint`,  so the  `dataFilesPerCheckpoint.tailMap(maxCommittedCheckpointId, false);`  will contains all the uncommitted data files,  include the files from current checkpoint.
   
   About your described process: 
   
   ```java
   Map<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.tailMap(checkpointId, true);
   ```
   
   This seems won't  commit all the previous files which have committed with failure .  How about the data files between (maxCommittedCheckpointId, checkpointId) ? 
   
   In my thought,  the current process should be correct.  If there're some places I misunderstood, pls correct me,thanks.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478775132



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    tableLoader.open(hadoopConf.get());

Review comment:
       Nevermind, looks like this is done in `dispose()` -- here's a good example of where prefixing with `this.` would be more clear.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] YesOrNo828 commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
YesOrNo828 commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r467840043



##########
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:
       I think this method `parseMaxCommittedCheckpointId` should be invoked when this job is stored.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479431162



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java
##########
@@ -34,15 +35,16 @@
   private static final long serialVersionUID = 1L;
 
   private final String fullTableName;
+  private final TaskWriterFactory<T> taskWriterFactory;
 
-  private transient TaskWriterFactory<T> taskWriterFactory;
   private transient TaskWriter<T> writer;
   private transient int subTaskId;
   private transient int attemptId;
 
   IcebergStreamWriter(String fullTableName, TaskWriterFactory<T> taskWriterFactory) {
     this.fullTableName = fullTableName;
     this.taskWriterFactory = taskWriterFactory;
+    setChainingStrategy(ChainingStrategy.ALWAYS);

Review comment:
       Thanks! Nice to learn more about how Flink works.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479589663



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(checkpointId);
+      this.maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    AppendFiles appendFiles = table.newAppend();
+    pendingDataFiles.forEach(appendFiles::appendFile);
+    appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+    appendFiles.set(FLINK_JOB_ID, flinkJobId);
+    appendFiles.commit();
+
+    // Clear the committed data files from dataFilesPerCheckpoint.
+    pendingFileMap.clear();
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFile> element) {
+    this.dataFilesOfCurrentCheckpoint.add(element.getValue());

Review comment:
       this probably gets complicated when we allow concurrent checkpoints. Committer can receive files from both checkpoint N and N+1. We probably need add the checkpointId in the DataFile. It might make sense to provide `FlinkDataFile` wrapper so that we can add Flink additional metadata.
   
   We have `FlinkDataFile` in our implementation for transmitting low and high timestamps. Now thinking about this issue. Maybe we can include checkpointId too so that committer can distinguish data files from different checkpoints.
   ```
   public class FlinkDataFile implements Serializable {
     private final long lowWatermark;
     private final long highWatermark;
     private final DataFile dataFile;
   ```
   
   This does imposes additional requirement on the writer. It needs to know the next checkpointId. I couldn't find the checkpointId from the context of the `initializeState(context)` method for either `AbstractStreamOperator` or `RichSinkFunction`. It will be nice if it can be exposed. 
   
   Alternatively, we can store the nextCheckpointId in the operator state.
   * operator list state can't deal with rescale as new subtasks won't get state
   * operator union list state is not scalable. Kafka source is suffering the scalability issue with union state.
   
   Note that we may flush file before checkpoint barrier comes. E.g. As I mentioned in another comment, we have Flink streaming jobs running in 3 regions in AWS and data warehouses live only in us-east-1. There is a backend service monitor files in two other remote regions and lift/copy them back to the us-east-1 home region. S3 cross-region file copy has 5 GB limit. AFor that reason, we flushes files if the size reaches 4 GB




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r473764143



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+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 GLOBAL_FILES_COMMITTER_UID = "flink.files-committer.uid";
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed.checkpoint.id";
+
+  private static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  // It will have an unique identifier for one job.
+  private final String filesCommitterUid;
+  private final String fullTableName;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  // 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 transient long maxCommittedCheckpointId;
+
+  // 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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+  private transient Table table;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(String filesCommitterUid, String fullTableName,
+                        Map<String, String> options, Configuration conf) {
+    this.filesCommitterUid = filesCommitterUid;
+    this.fullTableName = fullTableName;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(fullTableName, options, conf.get());
+
+    table = icebergCatalog.loadTable(TableIdentifier.parse(fullTableName));
+    maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, filesCommitterUid);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkArgument(maxCommittedCheckpointId > 0,

Review comment:
       I checked the [ZooKeeperCheckpointIDCounter](https://github.com/apache/flink/blob/39ef9593f4d88aafaa296f119d9f4a55c9731af1/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java#L82) and [StandaloneCheckpointIDCounter](https://github.com/apache/flink/blob/39ef9593f4d88aafaa296f119d9f4a55c9731af1/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java#L33).  Both of them should start from 1. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-675887605


   For this pull request, there are several issue I need to explain here: 
   1.  @JingsongLi suggest to consider the two cases: 1.  multiple flink jobs are writing the same table;  2. restart with a new job and continue to write the same time.  For the former case,  we will need a global id, such as job id or application id,  to identify the max checkpoint id we've committed for a given job. For example, we have two flink job: job1 and job2: 
     a.  `job1` commit the iceberg table with maxCheckpointId=1 ; 
     b.  the second `job2` commit the iceberg table with maxCheckpointId=5; 
     c.  `job1` commit the iceberg table with maxCheckpointId=2; 
     d.  the `job2` start to commit again, it need to find the maxCheckpointId corresponding to `job2`, which is 5 now. Then it need to rewrite its maxCheckpointId = 6 and commit the txn. 
   
   The global id of job will also help to resolve the restart issue2, because we will know that the newly started job is starting from checkpoint=1. 
   
   2.  the uid of operator (from @kbendick) issue,  I read the [document](https://ci.apache.org/projects/flink/flink-docs-stable/ops/upgrading.html#matching-operator-state) in flink. Indeed, it's used for state recover and need to be unique across jobs. 
   
   I've address this two issue in the new patch , also attached the unit tests. 
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478775492



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java
##########
@@ -34,15 +35,16 @@
   private static final long serialVersionUID = 1L;
 
   private final String fullTableName;
+  private final TaskWriterFactory<T> taskWriterFactory;
 
-  private transient TaskWriterFactory<T> taskWriterFactory;
   private transient TaskWriter<T> writer;
   private transient int subTaskId;
   private transient int attemptId;
 
   IcebergStreamWriter(String fullTableName, TaskWriterFactory<T> taskWriterFactory) {
     this.fullTableName = fullTableName;
     this.taskWriterFactory = taskWriterFactory;
+    setChainingStrategy(ChainingStrategy.ALWAYS);

Review comment:
       What does this do?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r480214713



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);

Review comment:
       I like the proposed solution. it should work.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-682267834


   Overall, I think this is ready to commit. I'd prefer to remove `Serializable` from the appender factory interface first, though. And it would be nice to get some of the other comments fixed or clarified (some are questions since I'm learning about how Flink works). Thanks @openinx!


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-678013408


   Rebased the master and did the things btw: 
   1.  add ORC as a case in the unit tests; 
   2. Since we've introduced the `CatalogLoader` and `TableLoader`, then  we could just use it inside `IcebergFilesCommitter` to load iceberg table lazily,  that looks more reasonable. 


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478772412



##########
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:
       I think in the Netflix version, we track old checkpoint state by writing a manifest file and appending that manifest to the table. The advantage is that state is really small so the job can be blocked from committing for a long time (days) and can recover when connectivity to commit has been restored.
   
   We use this to handle resilience when AWS regions can't talk to one another and our metastore is in a single region.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478790684



##########
File path: core/src/main/java/org/apache/iceberg/io/FileAppenderFactory.java
##########
@@ -19,14 +19,15 @@
 
 package org.apache.iceberg.io;
 
+import java.io.Serializable;
 import org.apache.iceberg.FileFormat;
 
 /**
  * Factory to create a new {@link FileAppender} to write records.
  *
  * @param <T> data type of the rows to append.
  */
-public interface FileAppenderFactory<T> {
+public interface FileAppenderFactory<T> extends Serializable {

Review comment:
       That sounds reasonable. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);

Review comment:
       It will be quite easy to support other object models, we only need a `MapFunction` to convert the generic object to `RowData` and define what's the `TypeInformation` for `RowData`.  For example, we could create a separate builder interfaces for the generic object models on top of this builder: 
   
   ```java
     public static <T> Builder builderFor(DataStream<T> input,
                                          Function<T, RowData> converter,
                                          TypeInformation<RowData> outputType) {
       DataStream<RowData> dataStream = input.map(converter::apply, outputType);
       return forRowData(dataStream);
     }
   ```

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);
+  }
+
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<Row> rowInput = null;
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder forRow(DataStream<Row> newRowInput) {
+      this.rowInput = newRowInput;
+      return this;
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    private DataStream<RowData> convert() {
+      Preconditions.checkArgument(rowInput != null, "The DataStream<Row> to convert shouldn't be null");
+
+      RowType rowType;
+      DataType[] fieldDataTypes;
+      if (tableSchema != null) {
+        rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+        fieldDataTypes = tableSchema.getFieldDataTypes();
+      } else {
+        rowType = FlinkSchemaUtil.convert(table.schema());
+        fieldDataTypes = TypeConversions.fromLogicalToDataType(rowType.getChildren().toArray(new LogicalType[0]));
+      }
+
+      DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+
+      return rowInput.map(rowConverter::toInternal, RowDataTypeInfo.of(rowType));
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowInput != null || rowDataInput != null,
+          "Should initialize input DataStream first with DataStream<Row> or DataStream<RowData>");

Review comment:
       OK, let me make those more clear. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java
##########
@@ -34,15 +35,16 @@
   private static final long serialVersionUID = 1L;
 
   private final String fullTableName;
+  private final TaskWriterFactory<T> taskWriterFactory;
 
-  private transient TaskWriterFactory<T> taskWriterFactory;
   private transient TaskWriter<T> writer;
   private transient int subTaskId;
   private transient int attemptId;
 
   IcebergStreamWriter(String fullTableName, TaskWriterFactory<T> taskWriterFactory) {
     this.fullTableName = fullTableName;
     this.taskWriterFactory = taskWriterFactory;
+    setChainingStrategy(ChainingStrategy.ALWAYS);

Review comment:
       Copied the javadoc from ChainingStrategy here, it's used for chaining the operators to be co-located on the same thread for better performance.
   ```java
   /**
    * Defines the chaining scheme for the operator. When an operator is chained to the
    * predecessor, it means that they run in the same thread. They become one operator
    * consisting of multiple steps.
    *
    * <p>The default value used by the StreamOperator is {@link #HEAD}, which means that
    * the operator is not chained to its predecessor. Most operators override this with
    * {@link #ALWAYS}, meaning they will be chained to predecessors whenever possible.
    */
   @PublicEvolving
   public enum ChainingStrategy {
   
   	/**
   	 * Operators will be eagerly chained whenever possible.
   	 *
   	 * <p>To optimize performance, it is generally a good practice to allow maximal
   	 * chaining and increase operator parallelism.
   	 */
   	ALWAYS,
   
   	/**
   	 * The operator will not be chained to the preceding or succeeding operators.
   	 */
   	NEVER,
   
   	/**
   	 * The operator will not be chained to the predecessor, but successors may chain to this
   	 * operator.
   	 */
   	HEAD
   }
   ```

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);
+  }
+
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<Row> rowInput = null;
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder forRow(DataStream<Row> newRowInput) {
+      this.rowInput = newRowInput;
+      return this;
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    private DataStream<RowData> convert() {
+      Preconditions.checkArgument(rowInput != null, "The DataStream<Row> to convert shouldn't be null");

Review comment:
       I think it could be removed now, because we've checked the nullable in the `Builder#build`. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r478798549



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java
##########
@@ -34,15 +35,16 @@
   private static final long serialVersionUID = 1L;
 
   private final String fullTableName;
+  private final TaskWriterFactory<T> taskWriterFactory;
 
-  private transient TaskWriterFactory<T> taskWriterFactory;
   private transient TaskWriter<T> writer;
   private transient int subTaskId;
   private transient int attemptId;
 
   IcebergStreamWriter(String fullTableName, TaskWriterFactory<T> taskWriterFactory) {
     this.fullTableName = fullTableName;
     this.taskWriterFactory = taskWriterFactory;
+    setChainingStrategy(ChainingStrategy.ALWAYS);

Review comment:
       The chaining strategy definition:
   ```
    * Defines the chaining scheme for the operator. When an operator is chained to the
    * predecessor, it means that they run in the same thread. They become one operator
    * consisting of multiple steps.
    *
    * <p>The default value used by the StreamOperator is {@link #HEAD}, which means that
    * the operator is not chained to its predecessor. Most operators override this with
    * {@link #ALWAYS}, meaning they will be chained to predecessors whenever possible.
   ```
   
   We should also set the chaining strategy of the writer, so as to avoid the network crossing, which will bring additional serialization cost.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
kbendick commented on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-665991975


   Given that https://github.com/apache/iceberg/pull/1180 and https://github.com/apache/iceberg/pull/1175 have been merged, would it be possible for you to merge those commits into this branch to make it easier to review? Or does the dependency on https://github.com/apache/iceberg/pull/1145 make that somewhat burdensome?
   
   Not a major issue, but it would help me in being able to just review / look through the code that's being proposed for addition and not the code that has already been merged in.
   
   If not, no worries. Appreciate all of the work it seems you've put into getting Flink going with Iceberg @openinx. I can see you've submitted quite a number of PRs on the matter so I'm especially looking to learn from you by reading and reviewing these PRs 👍 


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479862208



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)

Review comment:
       That sounds good to me,   it's necessary to support multiple iceberg sinks in the same job, I will open an issue and provide a patch with unit tests to address this thing. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479536903



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

Review comment:
       nit: maybe just "IcebergCommitter". In Iceberg, we can only commit files.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r471203934



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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 static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  private final String path;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  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, Map<String, String> options, Configuration conf) {
+    this.path = path;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(path, options, conf.get());
+    table = icebergCatalog.loadTable(TableIdentifier.parse(path));
+    maxCommittedCheckpointId = parseMaxCommittedCheckpointId(table.currentSnapshot());
+
+    dataFilesPerCheckpoint = Maps.newTreeMap();
+    dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      dataFilesPerCheckpoint = deserializeState(checkpointsState.get().iterator().next());
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(serializeState(dataFilesPerCheckpoint)));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.tailMap(maxCommittedCheckpointId, false);

Review comment:
       According to the current code, the data of the current checkpoint needs to be committed in the next cycle.
   This should not be the ideal way, which can lead to excessive delay.
   
   ```
   if (maxCommittedCheckpointId >= checkpointId) {
     return;
   }
   Map<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.tailMap(checkpointId, true);
   List<DataFile> pendingDataFiles = Lists.newArrayList();
   for (List<DataFile> dataFiles : pendingFileMap.values()) {
         pendingDataFiles.addAll(dataFiles);
   }
   -- Do AppendFiles committing
   maxCommittedCheckpointId = checkpointId;
   pendingFileMap.clear();
   ```
   This is the process I imagined.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi edited a comment on pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi edited a comment on pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#issuecomment-673357802


   @openinx and @rdblue I think we can consider https://github.com/apache/iceberg/pull/1332 first, what do you think?


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479515438



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(checkpointId);
+      this.maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
+
+    List<DataFile> pendingDataFiles = Lists.newArrayList();
+    for (List<DataFile> dataFiles : pendingFileMap.values()) {
+      pendingDataFiles.addAll(dataFiles);
+    }
+
+    AppendFiles appendFiles = table.newAppend();
+    pendingDataFiles.forEach(appendFiles::appendFile);
+    appendFiles.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+    appendFiles.set(FLINK_JOB_ID, flinkJobId);
+    appendFiles.commit();
+
+    // Clear the committed data files from dataFilesPerCheckpoint.
+    pendingFileMap.clear();
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFile> element) {
+    this.dataFilesOfCurrentCheckpoint.add(element.getValue());
+  }
+
+  @Override
+  public void endInput() {
+    // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly.
+    dataFilesPerCheckpoint.put(Long.MAX_VALUE, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+    dataFilesOfCurrentCheckpoint.clear();
+
+    commitUpToCheckpoint(Long.MAX_VALUE);
+  }
+
+  @Override
+  public void dispose() throws Exception {
+    if (tableLoader != null) {
+      tableLoader.close();
+    }
+  }
+
+  private static ListStateDescriptor<SortedMap<Long, List<DataFile>>> buildStateDescriptor() {
+    Comparator<Long> longComparator = Comparators.forType(Types.LongType.get());
+    // Construct a ListTypeInfo.
+    ListTypeInfo<DataFile> dataFileListTypeInfo = new ListTypeInfo<>(TypeInformation.of(DataFile.class));
+    // Construct a SortedMapTypeInfo.
+    SortedMapTypeInfo<Long, List<DataFile>> sortedMapTypeInfo = new SortedMapTypeInfo<>(
+        BasicTypeInfo.LONG_TYPE_INFO, dataFileListTypeInfo, longComparator
+    );
+    return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo);
+  }
+
+  static long getMaxCommittedCheckpointId(Table table, String flinkJobId) {

Review comment:
       nice. this is the reverse iteration I was looking for. we were just walking through the table.snapshots() iterable.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r460361491



##########
File path: build.gradle
##########
@@ -253,6 +255,9 @@ project(':iceberg-flink') {
     compileOnly("org.apache.hadoop:hadoop-minicluster") {
       exclude group: 'org.apache.avro', module: 'avro'
     }
+    compileOnly("de.javakaffee:kryo-serializers:0.45") {
+      exclude group: 'com.esotericsoftware', module: 'kryo'
+    }

Review comment:
       As discussed in https://github.com/apache/iceberg/pull/1145/files#diff-22e0fba47887f8c06bc56a654fd3ee9fL363, we were considering avoiding the usage of `de.javakaffee` for additional kryo serializers.
   
   Could we create custom Flink serializers, or possibly use POJOs or Flink's built in support for Avro serializers. There's a recently posted blog post on the discussion of serializer choices in Flink here: https://flink.apache.org/news/2020/04/15/flink-serialization-tuning-vol-1.html
   
   I think Avro makes the most sense given that it's been native to Flink for a long time and because Avro is so widely used in Iceberg already. 
   
   As an added bonus, Avro supports full schema evolution of state (such that changes to the Avro schema of types stored in stateful operators don't cause said operators to lose their state on upgrade to the new schema, provided they meet certain specifications which are relatively normal rules for type migrations, especially with Avro).




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r460362464



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergDataStream.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.util.UUID;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+
+public class IcebergDataStream {
+
+  private static final String ICEBERG_STREAM_WRITER = "Iceberg-Stream-Writer";
+  private static final String ICEBERG_FILES_COMMITTER = "Iceberg-Files-Committer";
+
+  private final DataStream<Row> dataStream;
+  private final String path;
+  private final Configuration conf;
+  private final TableSchema tableSchema;
+  private final Integer parallelism;
+
+  private IcebergDataStream(DataStream<Row> dataStream, String path,
+                            Configuration conf, TableSchema tableSchema,
+                            Integer parallelism) {
+    this.dataStream = dataStream;
+    this.path = path;
+    this.conf = conf;
+    this.tableSchema = tableSchema;
+    this.parallelism = parallelism;
+  }
+
+  public static Builder buildFor(DataStream<Row> dataStream) {
+    return new Builder().dataStream(dataStream);
+  }
+
+  public static class Builder {
+    private DataStream<Row> dataStream;
+    private String path;
+    private Configuration conf;
+    private TableSchema tableSchema;
+    private Integer parallelism;
+
+    private Builder dataStream(DataStream<Row> newDataStream) {
+      this.dataStream = newDataStream;
+      return this;
+    }
+
+    public Builder path(String newPath) {
+      this.path = newPath;
+      return this;
+    }
+
+    public Builder config(Configuration newConfig) {
+      this.conf = newConfig;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    public Builder parallelism(int newParallelism) {
+      this.parallelism = newParallelism;
+      return this;
+    }
+
+    public IcebergDataStream build() {
+      return new IcebergDataStream(dataStream, path, conf, tableSchema, parallelism);
+    }
+  }
+
+  public void append() {
+    IcebergStreamWriter streamWriter = IcebergStreamWriter.createStreamWriter(path, tableSchema, conf);
+    IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(path, conf);
+
+    SingleOutputStreamOperator<DataFile> operator = dataStream
+        .transform(ICEBERG_STREAM_WRITER, DataFileTypeInfo.TYPE_INFO, streamWriter)
+        .uid(UUID.randomUUID().toString());
+
+    if (parallelism != null && parallelism > 0) {
+      operator.setParallelism(parallelism);
+    }
+
+    operator.addSink(filesCommitter)
+        .name(ICEBERG_FILES_COMMITTER)
+        .uid(UUID.randomUUID().toString())

Review comment:
       Most of the time, I simply assign the `name` as the `uid`. In Scala, I use an implicit class to assign name and uid at the same time with the same value. This assumes that the name of an operator is unique, but so far I've not run into issues with the handful of jobs that I oversee.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] YesOrNo828 commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
YesOrNo828 commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r472816697



##########
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:
       Do you have any idea about if the snapshots were removed by another process, and this process commit a new snapshot to iceberg table, the flink committer will not get the correct `maxCommittedCheckpointId `.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r471203934



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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 static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  private final String path;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  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, Map<String, String> options, Configuration conf) {
+    this.path = path;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(path, options, conf.get());
+    table = icebergCatalog.loadTable(TableIdentifier.parse(path));
+    maxCommittedCheckpointId = parseMaxCommittedCheckpointId(table.currentSnapshot());
+
+    dataFilesPerCheckpoint = Maps.newTreeMap();
+    dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      dataFilesPerCheckpoint = deserializeState(checkpointsState.get().iterator().next());
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(serializeState(dataFilesPerCheckpoint)));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.tailMap(maxCommittedCheckpointId, false);

Review comment:
       According to the current code, the data of the current checkpoint needs to be committed in the next cycle.
   This should not be the ideal way, which can lead to excessive delay.
   
   ```
   if (maxCommittedCheckpointId >= checkpointId) {
     return;
   }
   Map<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.headMap(checkpointId, true);
   List<DataFile> pendingDataFiles = Lists.newArrayList();
   for (List<DataFile> dataFiles : pendingFileMap.values()) {
         pendingDataFiles.addAll(dataFiles);
   }
   -- Do AppendFiles committing
   maxCommittedCheckpointId = checkpointId;
   pendingFileMap.clear();
   ```
   This is the process I imagined.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r480208355



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())
+          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)
+          .setParallelism(1)
+          .setMaxParallelism(1);
+
+      return returnStream.addSink(new DiscardingSink())

Review comment:
       @openinx thx. that does answer my question. 
   
   Still adding a DiscardingSink may confuse the users. It seems that we really need a unified/improved sink interface (similar to FLIP-27) to supported bounded stream in sink function.
   
   What about any other Flink sinks used by bounded streaming/batch job? Do they all have to go through this model? 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r470395648



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergFilesCommitter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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 static final FlinkCatalogFactory CATALOG_FACTORY = new FlinkCatalogFactory();
+
+  private final String path;
+  private final SerializableConfiguration conf;
+  private final ImmutableMap<String, String> options;
+
+  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, Map<String, String> options, Configuration conf) {
+    this.path = path;
+    this.options = ImmutableMap.copyOf(options);
+    this.conf = new SerializableConfiguration(conf);
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    Catalog icebergCatalog = CATALOG_FACTORY.buildIcebergCatalog(path, options, conf.get());
+    table = icebergCatalog.loadTable(TableIdentifier.parse(path));
+    maxCommittedCheckpointId = parseMaxCommittedCheckpointId(table.currentSnapshot());
+
+    dataFilesPerCheckpoint = Maps.newTreeMap();
+    dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+    checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      dataFilesPerCheckpoint = deserializeState(checkpointsState.get().iterator().next());
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, ImmutableList.copyOf(dataFilesOfCurrentCheckpoint));
+
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.addAll(ImmutableList.of(serializeState(dataFilesPerCheckpoint)));
+
+    // Clear the local buffer for current checkpoint.
+    dataFilesOfCurrentCheckpoint.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {
+    NavigableMap<Long, List<DataFile>> pendingFileMap = dataFilesPerCheckpoint.tailMap(maxCommittedCheckpointId, false);

Review comment:
       I've added few comments about those members here https://github.com/apache/iceberg/pull/1185/commits/6b43ec5b4183d708920e07a4378b8cb6a8ad290b.   
   
   > Why exclude maxCommittedCheckpointId and checkpointId?
   
   The `maxCommittedCheckPointId` 's data files have been committed to iceberg,  so no need to commit them again, otherwise the table will have redundant datas.  For current `checkpointId`,  we've flushed the `dataFilesOfCurrentCheckpoint` into `dataFilesPerCheckpoint` in `snapshotState` method,  so we did not exclude the files for current checkpointId. 
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479951122



##########
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();
+
+  // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<DataFile> dataFilesOfCurrentCheckpoint = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient long maxCommittedCheckpointId;
+
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf) {
+    this.tableLoader = tableLoader;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open(hadoopConf.get());
+    this.table = tableLoader.loadTable();
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    if (context.isRestored()) {
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
+      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
+      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
+      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
+      // the iceberg table.
+      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,
+          "There should be an existing iceberg snapshot for current flink job: %s", flinkJobId);
+
+      SortedMap<Long, List<DataFile>> restoredDataFiles = checkpointsState.get().iterator().next();
+      // Only keep the uncommitted data files in the cache.
+      this.dataFilesPerCheckpoint.putAll(restoredDataFiles.tailMap(maxCommittedCheckpointId + 1));

Review comment:
       Committing those uncommitted data files here immediately sounds good to me,  it should won't impact the correctness here. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479536903



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

Review comment:
       nit: maybe just "IcebergCommitter". In Iceberg, we can only commit files.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org