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


##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.OpenContext;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+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.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.flink.maintenance.operator.TableChange;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommittableToTableChangeConverter
+    extends ProcessFunction<CommittableMessage<IcebergCommittable>, 
TableChange>
+    implements CheckpointedFunction {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CommittableToTableChangeConverter.class);
+
+  private FileIO io;
+  private String tableName;
+  private Map<Integer, PartitionSpec> specs;
+  private transient ListState<Tuple2<Long, String>> manifestFilesCommitedState;
+  private transient Set<String> manifestFilesCommitedSet;
+  private transient NavigableMap<Long, List<String>> commitRequestMap;
+  private transient List<Tuple2<Long, String>> manifestFilesCommitedList;
+  private transient String flinkJobId;
+  // Maximum number of manifests to be committed at a time.
+  // It is hardcoded for now, we can revisit in the future if config is needed.
+  private int maxSize = 1000;
+
+  public CommittableToTableChangeConverter(
+      FileIO fileIO, String tableName, Map<Integer, PartitionSpec> specs) {
+    Preconditions.checkNotNull(fileIO, "FileIO should not be null");
+    Preconditions.checkNotNull(tableName, "TableName should not be null");
+    Preconditions.checkNotNull(specs, "Specs should not be null");
+    this.io = fileIO;
+    this.tableName = tableName;
+    this.specs = specs;
+  }
+
+  @VisibleForTesting
+  CommittableToTableChangeConverter(
+      FileIO fileIO, String tableName, Map<Integer, PartitionSpec> specs, int 
maxSize) {
+    Preconditions.checkNotNull(fileIO, "FileIO should not be null");
+    Preconditions.checkNotNull(tableName, "TableName should not be null");
+    Preconditions.checkNotNull(specs, "Specs should not be null");
+    this.io = fileIO;
+    this.tableName = tableName;
+    this.specs = specs;
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws 
Exception {
+    this.manifestFilesCommitedSet = Sets.newHashSet();
+    this.manifestFilesCommitedList = Lists.newArrayList();
+    this.commitRequestMap = Maps.newTreeMap();
+    this.manifestFilesCommitedState =
+        context
+            .getOperatorStateStore()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "manifests-commited", TypeInformation.of(new TypeHint<>() 
{})));
+    if (context.isRestored()) {
+      for (Tuple2<Long, String> checkPointIdAndManifestTuple : 
manifestFilesCommitedState.get()) {
+        manifestFilesCommitedSet.add(checkPointIdAndManifestTuple.f1);
+        manifestFilesCommitedList.add(checkPointIdAndManifestTuple);
+        manifestFilesCommitedList.forEach(
+            tuple ->
+                commitRequestMap
+                    .computeIfAbsent(tuple.f0, k -> Lists.newArrayList())
+                    .add(tuple.f1));
+      }
+    }
+  }
+
+  @Override
+  public void open(OpenContext openContext) throws Exception {
+    super.open(openContext);
+    Preconditions.checkState(
+        getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks() == 1,
+        "CommittableToTableChangeConverter must run with parallelism 1, 
current parallelism: %s",
+        getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks());
+
+    this.flinkJobId = getRuntimeContext().getJobId().toString();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    manifestFilesCommitedState.clear();
+    manifestFilesCommitedState.addAll(manifestFilesCommitedList);
+  }
+
+  @Override
+  public void processElement(
+      CommittableMessage<IcebergCommittable> value,
+      ProcessFunction<CommittableMessage<IcebergCommittable>, 
TableChange>.Context ctx,
+      Collector<TableChange> out)
+      throws Exception {
+    if (value instanceof CommittableWithLineage) {
+      IcebergCommittable committable =
+          ((CommittableWithLineage<IcebergCommittable>) 
value).getCommittable();
+
+      if (committable == null || committable.manifest().length == 0) {

Review Comment:
   My bad.
   I wasn't sure what we do, when we get an empty writer result, and not 
reached the `MAX_CONTINUOUS_EMPTY_COMMITS`. Turns out, then we send a 
committable where the `committable.manifest().length == 0`. So your code was 
correct.
   
   In this case it is fine, not to emit a `TableChange`, as we don't want to 
run the compaction if there are no commits to compact.
   
   So again, sorry for the confusion. Your original code was fine!
   Could you please put this back:
   ```
         if (committable == null || committable.manifest().length == 0) {
           return;
         }
   ```
   
   Thanks,
   Peter



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

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


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

Reply via email to