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


##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java:
##########
@@ -216,7 +237,81 @@ public SimpleVersionedSerializer<IcebergCommittable> 
getCommittableSerializer()
   @Override
   public void addPostCommitTopology(
       DataStream<CommittableMessage<IcebergCommittable>> committables) {
-    // TODO Support small file compaction
+
+    if (!compactMode) {
+      return;
+    }
+
+    SingleOutputStreamOperator<TableChange> tableChangeStream =
+        committables
+            .global()
+            .transform(
+                "CommittableToTableChangeConverter",
+                TypeInformation.of(TableChange.class),
+                new CommittableToTableChangeConverter(tableLoader.clone()))
+            .uid("committable-to-table-change-converter")
+            .forceNonParallel();
+    try {
+      TriggerLockFactory triggerLockFactory = 
LockFactoryCreator.create(flinkOptions);
+      RewriteDataFiles.Builder rewriteBuilder = 
createRewriteBuilder(flinkOptions);
+      TableMaintenance.Builder builder =
+          TableMaintenance.forChangeStream(tableChangeStream, tableLoader, 
triggerLockFactory)
+              .add(rewriteBuilder);
+      configureTableMaintenance(builder, flinkOptions);
+      builder.append();
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to create tableMaintenance ", e);
+    }
+  }
+
+  private RewriteDataFiles.Builder createRewriteBuilder(Map<String, String> 
properties) {
+    RewriteDataFilesConfig rewriteDataFilesConfig = new 
RewriteDataFilesConfig(properties);
+    RewriteDataFiles.Builder builder = RewriteDataFiles.builder();
+    Optional.ofNullable(rewriteDataFilesConfig.getPartialProgressEnable())
+        .ifPresent(builder::partialProgressEnabled);
+    Optional.ofNullable(rewriteDataFilesConfig.getPartialProgressMaxCommits())
+        .ifPresent(builder::partialProgressMaxCommits);
+    Optional.ofNullable(rewriteDataFilesConfig.getMaxRewriteBytes())
+        .ifPresent(builder::maxRewriteBytes);
+    Optional.ofNullable(rewriteDataFilesConfig.getTargetFileSizeBytes())
+        .ifPresent(builder::targetFileSizeBytes);
+    Optional.ofNullable(rewriteDataFilesConfig.getMinFileSizeBytes())
+        .ifPresent(builder::minFileSizeBytes);
+    Optional.ofNullable(rewriteDataFilesConfig.getMaxFileSizeBytes())
+        .ifPresent(builder::maxFileSizeBytes);
+    Optional.ofNullable(rewriteDataFilesConfig.getMaxFileGroupSizeBytes())
+        .ifPresent(builder::maxFileGroupSizeBytes);
+    Optional.ofNullable(rewriteDataFilesConfig.getMinInputFiles())
+        .ifPresent(builder::minInputFiles);
+    Optional.ofNullable(rewriteDataFilesConfig.getDeleteFileThreshold())
+        .ifPresent(builder::deleteFileThreshold);
+    
Optional.ofNullable(rewriteDataFilesConfig.getRewriteAll()).ifPresent(builder::rewriteAll);
+
+    Optional.ofNullable(rewriteDataFilesConfig.getScheduleOnCommitCount())
+        .ifPresent(builder::scheduleOnCommitCount);
+    Optional.ofNullable(rewriteDataFilesConfig.getScheduleOnDataFileCount())
+        .ifPresent(builder::scheduleOnDataFileCount);
+    Optional.ofNullable(rewriteDataFilesConfig.getScheduleOnDataFileSize())
+        .ifPresent(builder::scheduleOnDataFileSize);
+    Optional.ofNullable(rewriteDataFilesConfig.getScheduleOnIntervalSecond())
+        .ifPresent(
+            intervalSecond -> 
builder.scheduleOnInterval(Duration.ofSeconds(intervalSecond)));
+
+    return builder;
+  }

Review Comment:
   Feels like all this configuration logic doesn't belong in the IcebergSink 
but should be part of the RewriteDataFiles/Builder. 
   
   That way we can reuse the configuration, or the user can use the same 
options to configure standalone table maintenance as with the post commit 
topology



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.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.IOException;
+import java.util.List;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.operator.TableChange;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommittableToTableChangeConverter extends 
AbstractStreamOperator<TableChange>
+    implements OneInputStreamOperator<CommittableMessage<IcebergCommittable>, 
TableChange> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CommittableToTableChangeConverter.class);
+
+  private final TableLoader tableLoader;
+  private Table table;
+  private transient ListState<ManifestFile> toRemoveManifestFileState;

Review Comment:
   should we call this `manifestFilesToRemoveState`?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.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.IOException;
+import java.util.List;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.operator.TableChange;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommittableToTableChangeConverter extends 
AbstractStreamOperator<TableChange>
+    implements OneInputStreamOperator<CommittableMessage<IcebergCommittable>, 
TableChange> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CommittableToTableChangeConverter.class);
+
+  private final TableLoader tableLoader;
+  private Table table;

Review Comment:
   should be transient as well?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.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.IOException;
+import java.util.List;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.operator.TableChange;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommittableToTableChangeConverter extends 
AbstractStreamOperator<TableChange>
+    implements OneInputStreamOperator<CommittableMessage<IcebergCommittable>, 
TableChange> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CommittableToTableChangeConverter.class);
+
+  private final TableLoader tableLoader;
+  private Table table;
+  private transient ListState<ManifestFile> toRemoveManifestFileState;
+  private transient List<ManifestFile> toRemoveManifestFileList;
+  private transient long lastCompletedCheckpointId = -1L;
+  private transient String flinkJobId;
+
+  public CommittableToTableChangeConverter(TableLoader tableLoader) {
+    Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+    this.tableLoader = tableLoader;
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws 
Exception {
+    super.initializeState(context);
+    this.toRemoveManifestFileList = Lists.newArrayList();
+    this.flinkJobId = 
getContainingTask().getEnvironment().getJobID().toString();
+    this.toRemoveManifestFileState =
+        context
+            .getOperatorStateStore()
+            .getListState(new ListStateDescriptor<>("to-remove-manifest", 
ManifestFile.class));
+    if (context.isRestored()) {
+      toRemoveManifestFileList = 
Lists.newArrayList(toRemoveManifestFileState.get());

Review Comment:
   should we call this `manifests-to-remove` and `manifestFilesToRemove` 
respectively?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.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.IOException;
+import java.util.List;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.operator.TableChange;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommittableToTableChangeConverter extends 
AbstractStreamOperator<TableChange>
+    implements OneInputStreamOperator<CommittableMessage<IcebergCommittable>, 
TableChange> {

Review Comment:
   Why is this implemented as a OneInputStreamOperator? Can this be a simple 
ProcessFunction that implements the CheckpointedFunction interface for using 
non-keyed states?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.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.IOException;
+import java.util.List;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.operator.TableChange;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommittableToTableChangeConverter extends 
AbstractStreamOperator<TableChange>
+    implements OneInputStreamOperator<CommittableMessage<IcebergCommittable>, 
TableChange> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CommittableToTableChangeConverter.class);
+
+  private final TableLoader tableLoader;
+  private Table table;
+  private transient ListState<ManifestFile> toRemoveManifestFileState;
+  private transient List<ManifestFile> toRemoveManifestFileList;
+  private transient long lastCompletedCheckpointId = -1L;
+  private transient String flinkJobId;
+
+  public CommittableToTableChangeConverter(TableLoader tableLoader) {
+    Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+    this.tableLoader = tableLoader;
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws 
Exception {
+    super.initializeState(context);
+    this.toRemoveManifestFileList = Lists.newArrayList();
+    this.flinkJobId = 
getContainingTask().getEnvironment().getJobID().toString();
+    this.toRemoveManifestFileState =
+        context
+            .getOperatorStateStore()
+            .getListState(new ListStateDescriptor<>("to-remove-manifest", 
ManifestFile.class));
+    if (context.isRestored()) {
+      toRemoveManifestFileList = 
Lists.newArrayList(toRemoveManifestFileState.get());
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    if (!tableLoader.isOpen()) {
+      tableLoader.open();
+    }
+
+    this.table = tableLoader.loadTable();
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    toRemoveManifestFileState.update(toRemoveManifestFileList);
+  }
+
+  @Override
+  public void 
processElement(StreamRecord<CommittableMessage<IcebergCommittable>> record)
+      throws Exception {
+    if (record.getValue() instanceof CommittableWithLineage) {
+      CommittableWithLineage<IcebergCommittable> committable =
+          (CommittableWithLineage<IcebergCommittable>) record.getValue();
+      TableChange tableChange = 
convertToTableChange(committable.getCommittable());
+      output.collect(new StreamRecord<>(tableChange));
+    }
+  }
+
+  private TableChange convertToTableChange(IcebergCommittable 
icebergCommittable)
+      throws IOException {
+    if (icebergCommittable == null || icebergCommittable.manifest().length == 
0) {
+      return TableChange.empty();
+    }
+
+    DeltaManifests deltaManifests =
+        SimpleVersionedSerialization.readVersionAndDeSerialize(
+            DeltaManifestsSerializer.INSTANCE, icebergCommittable.manifest());
+    WriteResult writeResult =
+        FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), 
table.specs());
+    toRemoveManifestFileList.addAll(deltaManifests.manifests());
+
+    int dataFileCount = 0;
+    long dataFileSizeInBytes = 0L;
+    for (DataFile dataFile : writeResult.dataFiles()) {
+      dataFileCount++;
+      dataFileSizeInBytes += dataFile.fileSizeInBytes();
+    }
+
+    int posDeleteFileCount = 0;
+    long posDeleteRecordCount = 0L;
+    int eqDeleteFileCount = 0;
+    long eqDeleteRecordCount = 0L;
+
+    for (DeleteFile deleteFile : writeResult.deleteFiles()) {
+      switch (deleteFile.content()) {
+        case POSITION_DELETES:
+          posDeleteFileCount++;
+          posDeleteRecordCount += deleteFile.recordCount();
+          break;
+        case EQUALITY_DELETES:
+          eqDeleteFileCount++;
+          eqDeleteRecordCount += deleteFile.recordCount();
+          break;
+        default:
+          // Unexpected delete file types don't impact compaction task 
statistics, so ignore.
+          LOG.info("Unexpected delete file content:{}", deleteFile.content());

Review Comment:
   log on debug/warn maybe?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.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.IOException;
+import java.util.List;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.operator.TableChange;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommittableToTableChangeConverter extends 
AbstractStreamOperator<TableChange>
+    implements OneInputStreamOperator<CommittableMessage<IcebergCommittable>, 
TableChange> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CommittableToTableChangeConverter.class);
+
+  private final TableLoader tableLoader;
+  private Table table;
+  private transient ListState<ManifestFile> toRemoveManifestFileState;
+  private transient List<ManifestFile> toRemoveManifestFileList;
+  private transient long lastCompletedCheckpointId = -1L;
+  private transient String flinkJobId;
+
+  public CommittableToTableChangeConverter(TableLoader tableLoader) {
+    Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+    this.tableLoader = tableLoader;
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws 
Exception {
+    super.initializeState(context);
+    this.toRemoveManifestFileList = Lists.newArrayList();
+    this.flinkJobId = 
getContainingTask().getEnvironment().getJobID().toString();
+    this.toRemoveManifestFileState =
+        context
+            .getOperatorStateStore()
+            .getListState(new ListStateDescriptor<>("to-remove-manifest", 
ManifestFile.class));
+    if (context.isRestored()) {
+      toRemoveManifestFileList = 
Lists.newArrayList(toRemoveManifestFileState.get());
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    if (!tableLoader.isOpen()) {
+      tableLoader.open();
+    }
+
+    this.table = tableLoader.loadTable();
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    toRemoveManifestFileState.update(toRemoveManifestFileList);
+  }
+
+  @Override
+  public void 
processElement(StreamRecord<CommittableMessage<IcebergCommittable>> record)
+      throws Exception {
+    if (record.getValue() instanceof CommittableWithLineage) {
+      CommittableWithLineage<IcebergCommittable> committable =
+          (CommittableWithLineage<IcebergCommittable>) record.getValue();
+      TableChange tableChange = 
convertToTableChange(committable.getCommittable());
+      output.collect(new StreamRecord<>(tableChange));
+    }
+  }
+
+  private TableChange convertToTableChange(IcebergCommittable 
icebergCommittable)
+      throws IOException {
+    if (icebergCommittable == null || icebergCommittable.manifest().length == 
0) {
+      return TableChange.empty();
+    }
+
+    DeltaManifests deltaManifests =
+        SimpleVersionedSerialization.readVersionAndDeSerialize(
+            DeltaManifestsSerializer.INSTANCE, icebergCommittable.manifest());
+    WriteResult writeResult =
+        FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), 
table.specs());
+    toRemoveManifestFileList.addAll(deltaManifests.manifests());
+
+    int dataFileCount = 0;
+    long dataFileSizeInBytes = 0L;
+    for (DataFile dataFile : writeResult.dataFiles()) {
+      dataFileCount++;
+      dataFileSizeInBytes += dataFile.fileSizeInBytes();
+    }
+
+    int posDeleteFileCount = 0;
+    long posDeleteRecordCount = 0L;
+    int eqDeleteFileCount = 0;
+    long eqDeleteRecordCount = 0L;
+
+    for (DeleteFile deleteFile : writeResult.deleteFiles()) {
+      switch (deleteFile.content()) {
+        case POSITION_DELETES:
+          posDeleteFileCount++;
+          posDeleteRecordCount += deleteFile.recordCount();
+          break;
+        case EQUALITY_DELETES:
+          eqDeleteFileCount++;
+          eqDeleteRecordCount += deleteFile.recordCount();
+          break;
+        default:
+          // Unexpected delete file types don't impact compaction task 
statistics, so ignore.
+          LOG.info("Unexpected delete file content:{}", deleteFile.content());
+      }
+    }
+
+    TableChange tableChange =
+        TableChange.builder()
+            .dataFileCount(dataFileCount)
+            .dataFileSizeInBytes(dataFileSizeInBytes)
+            .posDeleteFileCount(posDeleteFileCount)
+            .posDeleteRecordCount(posDeleteRecordCount)
+            .eqDeleteRecordCount(eqDeleteRecordCount)
+            .eqDeleteFileCount(eqDeleteFileCount)
+            .commitCount(1)
+            .build();
+
+    return tableChange;
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    if (checkpointId > lastCompletedCheckpointId) {
+      this.lastCompletedCheckpointId = checkpointId;
+      FlinkManifestUtil.deleteCommittedManifests(
+          table, toRemoveManifestFileList, flinkJobId, checkpointId);
+      toRemoveManifestFileList.clear();
+      toRemoveManifestFileState.clear();
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    super.close();
+    if (tableLoader.isOpen()) {
+      tableLoader.close();
+    }

Review Comment:
   close / null out table object maybe?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFilesConfig.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.maintenance.api;
+
+import java.util.Map;
+import org.apache.iceberg.util.PropertyUtil;
+
+public class RewriteDataFilesConfig {
+  private static final String CONFIG_PREFIX = "flink-maintenance.rewrite.";
+
+  private final Map<String, String> properties;
+
+  public RewriteDataFilesConfig(Map<String, String> properties) {
+    this.properties = properties;
+  }
+
+  public static final String PARTIAL_PROGRESS_ENABLE = CONFIG_PREFIX + 
"partial-progress-enabled";
+
+  public static final String PARTIAL_PROGRESS_MAX_COMMITS =
+      CONFIG_PREFIX + "partial-progress-max-commits";
+
+  public static final String MAX_REWRITE_BYTES = CONFIG_PREFIX + "max-bytes";
+
+  public static final String TARGET_FILE_SIZE_BYTES = CONFIG_PREFIX + 
"target-file-size-bytes";
+
+  public static final String MIN_FILE_SIZE_BYTES = CONFIG_PREFIX + 
"min-file-size-bytes";

Review Comment:
   Feels like all if these config keys are duplicated from other places which 
introduces a lot of duplication and seems super error prone. Can we reuse the 
existing configs here somehow?



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