stevenzwu commented on code in PR #11497:
URL: https://github.com/apache/iceberg/pull/11497#discussion_r2045041541


##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.operator;
+
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.FileRewritePlan;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+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.math.IntMath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Plans the rewrite groups using the {@link BinPackRewriteFilePlanner}. The 
input is the {@link
+ * Trigger}, the output is zero, or some {@link PlannedGroup}s.
+ */
+@Internal
+public class DataFileRewritePlanner
+    extends ProcessFunction<Trigger, DataFileRewritePlanner.PlannedGroup> {
+  static final String MESSAGE_PREFIX = "[For table {} with {}[{}] at {}]: ";
+  static final String MESSAGE_FORMAT_PREFIX = "[For table %s with {%s}[{%d}] 
at {%d}]: ";
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewritePlanner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+  private final int partialProgressMaxCommits;
+  private final long maxRewriteBytes;
+  private final Map<String, String> rewriterOptions;
+  private transient Counter errorCounter;
+
+  public DataFileRewritePlanner(
+      String tableName,
+      String taskName,
+      int taskIndex,
+      TableLoader tableLoader,
+      int newPartialProgressMaxCommits,
+      long maxRewriteBytes,
+      Map<String, String> rewriterOptions) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(rewriterOptions, "Options map should no be 
null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+    this.partialProgressMaxCommits = newPartialProgressMaxCommits;
+    this.maxRewriteBytes = maxRewriteBytes;
+    this.rewriterOptions = rewriterOptions;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    tableLoader.open();
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+  }
+
+  @Override
+  public void processElement(Trigger value, Context ctx, 
Collector<PlannedGroup> out)
+      throws Exception {
+    LOG.debug(
+        DataFileRewritePlanner.MESSAGE_PREFIX + "Creating rewrite plan",
+        tableName,
+        taskName,
+        taskIndex,
+        ctx.timestamp());
+    try {
+      SerializableTable table =
+          (SerializableTable) 
SerializableTable.copyOf(tableLoader.loadTable());

Review Comment:
   where is the `TableLoader` opened and closed?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.operator;
+
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.FileRewritePlan;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+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.math.IntMath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Plans the rewrite groups using the {@link BinPackRewriteFilePlanner}. The 
input is the {@link
+ * Trigger}, the output is zero, or some {@link PlannedGroup}s.
+ */
+@Internal
+public class DataFileRewritePlanner
+    extends ProcessFunction<Trigger, DataFileRewritePlanner.PlannedGroup> {
+  static final String MESSAGE_PREFIX = "[For table {} with {}[{}] at {}]: ";
+  static final String MESSAGE_FORMAT_PREFIX = "[For table %s with {%s}[{%d}] 
at {%d}]: ";
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewritePlanner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+  private final int partialProgressMaxCommits;
+  private final long maxRewriteBytes;
+  private final Map<String, String> rewriterOptions;
+  private transient Counter errorCounter;
+
+  public DataFileRewritePlanner(
+      String tableName,
+      String taskName,
+      int taskIndex,
+      TableLoader tableLoader,
+      int newPartialProgressMaxCommits,
+      long maxRewriteBytes,
+      Map<String, String> rewriterOptions) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(rewriterOptions, "Options map should no be 
null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+    this.partialProgressMaxCommits = newPartialProgressMaxCommits;
+    this.maxRewriteBytes = maxRewriteBytes;
+    this.rewriterOptions = rewriterOptions;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    tableLoader.open();
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+  }
+
+  @Override
+  public void processElement(Trigger value, Context ctx, 
Collector<PlannedGroup> out)
+      throws Exception {
+    LOG.debug(

Review Comment:
   nit: info level?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.operator;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING;
+
+import java.util.Collections;
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import 
org.apache.iceberg.flink.maintenance.operator.DataFileRewritePlanner.PlannedGroup;
+import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory;
+import org.apache.iceberg.flink.sink.TaskWriterFactory;
+import org.apache.iceberg.flink.source.DataIterator;
+import org.apache.iceberg.flink.source.FileScanTaskReader;
+import org.apache.iceberg.flink.source.RowDataFileScanTaskReader;
+import org.apache.iceberg.io.TaskWriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Executes a rewrite for a single {@link PlannedGroup}. Reads the files with 
the standard {@link
+ * FileScanTaskReader}, so the delete files are considered, and writes using 
the {@link
+ * TaskWriterFactory}. The output is an {@link ExecutedGroup}.
+ */
+@Internal
+public class DataFileRewriteRunner
+    extends ProcessFunction<PlannedGroup, DataFileRewriteRunner.ExecutedGroup> 
{
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewriteRunner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+
+  private transient int subTaskId;
+  private transient int attemptId;
+  private transient Counter errorCounter;
+
+  public DataFileRewriteRunner(String tableName, String taskName, int 
taskIndex) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+  }
+
+  @Override
+  public void open(Configuration parameters) {
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+
+    this.subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask();
+    this.attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber();
+  }
+
+  @Override
+  public void processElement(PlannedGroup value, Context ctx, 
Collector<ExecutedGroup> out)
+      throws Exception {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Rewriting files {} from {}",

Review Comment:
   `Rewriting group {} with files {}`?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteCommitter.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.operator;
+
+import java.io.IOException;
+import java.util.Locale;
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Commits the rewrite changes using {@link RewriteDataFilesCommitManager}. 
The input is a {@link
+ * DataFileRewriteRunner.ExecutedGroup}. Only {@link Watermark} is emitted 
which is chained to
+ * {@link TaskResultAggregator} input 1.
+ */
+@Internal
+public class DataFileRewriteCommitter extends AbstractStreamOperator<Trigger>
+    implements OneInputStreamOperator<DataFileRewriteRunner.ExecutedGroup, 
Trigger> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewriteCommitter.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+
+  private transient Table table;
+  private transient CommitService commitService;
+  private transient int processed;
+  private transient Counter errorCounter;
+  private transient Counter addedDataFileNumCounter;
+  private transient Counter addedDataFileSizeCounter;
+  private transient Counter removedDataFileNumCounter;
+  private transient Counter removedDataFileSizeCounter;
+
+  public DataFileRewriteCommitter(
+      String tableName, String taskName, int taskIndex, TableLoader 
tableLoader) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+
+    tableLoader.open();
+    this.table = tableLoader.loadTable();
+
+    MetricGroup taskMetricGroup =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex);
+    this.errorCounter = 
taskMetricGroup.counter(TableMaintenanceMetrics.ERROR_COUNTER);
+    this.addedDataFileNumCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.ADDED_DATA_FILE_NUM_METRIC);
+    this.addedDataFileSizeCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.ADDED_DATA_FILE_SIZE_METRIC);
+    this.removedDataFileNumCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.REMOVED_DATA_FILE_NUM_METRIC);
+    this.removedDataFileSizeCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.REMOVED_DATA_FILE_SIZE_METRIC);
+
+    this.processed = 0;
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFileRewriteRunner.ExecutedGroup> 
streamRecord) {
+    DataFileRewriteRunner.ExecutedGroup executedGroup = 
streamRecord.getValue();
+    try {
+      if (commitService == null) {
+        FlinkRewriteDataFilesCommitManager commitManager =
+            new FlinkRewriteDataFilesCommitManager(
+                table, executedGroup.snapshotId(), 
streamRecord.getTimestamp());
+        this.commitService = 
commitManager.service(executedGroup.groupsPerCommit());
+        commitService.start();
+      }
+
+      commitService.offer(executedGroup.group());
+      ++processed;
+    } catch (Exception e) {
+      LOG.warn(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Exception processing {}",
+          tableName,
+          taskName,
+          taskIndex,
+          streamRecord.getTimestamp(),
+          executedGroup,
+          e);
+      output.collect(TaskResultAggregator.ERROR_STREAM, new StreamRecord<>(e));
+      errorCounter.inc();
+    }
+  }
+
+  @Override
+  public void processWatermark(Watermark mark) throws Exception {
+    try {
+      if (commitService != null) {
+        commitService.close();
+        if (processed != commitService.results().size()) {
+          throw new RuntimeException(
+              String.format(
+                  Locale.ROOT,
+                  DataFileRewritePlanner.MESSAGE_FORMAT_PREFIX
+                      + "From %d commits only %d were successful",
+                  tableName,
+                  taskName,
+                  taskIndex,
+                  mark.getTimestamp(),
+                  processed,
+                  commitService.results().size()));
+        }
+      }
+
+      LOG.info(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Successfully completed data 
file compaction",
+          tableName,
+          taskName,
+          taskIndex,
+          mark.getTimestamp());
+    } catch (Exception e) {
+      LOG.warn(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Exception closing commit 
service",
+          tableName,
+          taskName,
+          taskIndex,
+          mark.getTimestamp(),
+          e);
+      output.collect(TaskResultAggregator.ERROR_STREAM, new StreamRecord<>(e));
+      errorCounter.inc();
+    }
+
+    // Cleanup
+    this.commitService = null;
+    this.processed = 0;
+
+    super.processWatermark(mark);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (commitService != null) {
+      commitService.close();
+    }
+  }
+
+  private class FlinkRewriteDataFilesCommitManager extends 
RewriteDataFilesCommitManager {
+    private final long timestamp;
+
+    FlinkRewriteDataFilesCommitManager(Table table, long snapshotId, long 
timestamp) {
+      super(table, snapshotId);
+      this.timestamp = timestamp;
+    }
+
+    @Override
+    public void commitFileGroups(Set<RewriteFileGroup> fileGroups) {
+      super.commitFileGroups(fileGroups);
+      LOG.debug(

Review Comment:
   info level?



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java:
##########
@@ -0,0 +1,409 @@
+/*
+ * 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 static org.apache.iceberg.flink.SimpleDataUtil.createRecord;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.COMMIT_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.PLANNER_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.REWRITE_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ADDED_DATA_FILE_NUM_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ADDED_DATA_FILE_SIZE_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ERROR_COUNTER;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.REMOVED_DATA_FILE_NUM_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.REMOVED_DATA_FILE_SIZE_METRIC;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+import java.util.stream.StreamSupport;
+import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import 
org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.Test;
+
+class TestRewriteDataFiles extends MaintenanceTaskTestBase {
+  @Test
+  void testRewriteUnpartitioned() throws Exception {
+    Table table = createTable();
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+    insert(table, 3, "c");
+    insert(table, 4, "d");
+
+    appendRewriteDataFiles(
+        RewriteDataFiles.builder()
+            .parallelism(2)
+            .deleteFileThreshold(10)
+            .targetFileSizeBytes(1_000_000L)
+            .maxFileGroupSizeBytes(10_000_000L)
+            .maxFileSizeBytes(2_000_000L)
+            .minFileSizeBytes(500_000L)
+            .minInputFiles(2)
+            .partialProgressEnabled(true)
+            .partialProgressMaxCommits(1)
+            .maxRewriteBytes(100_000L)
+            .rewriteAll(false));
+
+    runAndWaitForSuccess(infra.env(), infra.source(), infra.sink());
+
+    assertFileNum(table, 1, 0);
+
+    SimpleDataUtil.assertTableRecords(
+        table,
+        ImmutableList.of(
+            createRecord(1, "a"),
+            createRecord(2, "b"),
+            createRecord(3, "c"),
+            createRecord(4, "d")));
+  }
+
+  @Test
+  void testRewritePartitioned() throws Exception {
+    Table table = createPartitionedTable();
+    insertPartitioned(table, 1, "p1");
+    insertPartitioned(table, 2, "p1");
+    insertPartitioned(table, 3, "p2");
+    insertPartitioned(table, 4, "p2");
+
+    appendRewriteDataFiles();
+
+    runAndWaitForSuccess(infra.env(), infra.source(), infra.sink());
+
+    assertFileNum(table, 2, 0);
+
+    SimpleDataUtil.assertTableRecords(
+        table,
+        ImmutableList.of(
+            createRecord(1, "p1"),
+            createRecord(2, "p1"),
+            createRecord(3, "p2"),
+            createRecord(4, "p2")));
+  }
+
+  @Test
+  void testFailure() throws Exception {
+    Table table = createTable();
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+
+    appendRewriteDataFiles();
+
+    runAndWaitForFailure(infra.env(), infra.source(), infra.sink());
+
+    // Check the metrics
+    MetricsReporterFactoryForTests.assertCounters(
+        new ImmutableMap.Builder<List<String>, Long>()
+            .put(
+                ImmutableList.of(
+                    PLANNER_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ERROR_COUNTER),
+                1L)

Review Comment:
   if the simulated failure is for planner only, maybe change this test name to 
`testPlannerFailure`?



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java:
##########
@@ -0,0 +1,409 @@
+/*
+ * 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 static org.apache.iceberg.flink.SimpleDataUtil.createRecord;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.COMMIT_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.PLANNER_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.REWRITE_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ADDED_DATA_FILE_NUM_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ADDED_DATA_FILE_SIZE_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ERROR_COUNTER;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.REMOVED_DATA_FILE_NUM_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.REMOVED_DATA_FILE_SIZE_METRIC;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+import java.util.stream.StreamSupport;
+import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import 
org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.Test;
+
+class TestRewriteDataFiles extends MaintenanceTaskTestBase {
+  @Test
+  void testRewriteUnpartitioned() throws Exception {
+    Table table = createTable();
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+    insert(table, 3, "c");
+    insert(table, 4, "d");

Review Comment:
   we should assert the file number to be 4 before compaction. this applies to 
other test methods as well



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java:
##########
@@ -124,12 +134,62 @@ protected static Table createTableWithDelete() {
             ImmutableMap.of("format-version", "2", "write.upsert.enabled", 
"true"));
   }
 
+  protected static Table createPartitionedTable() {
+    return CATALOG_EXTENSION
+        .catalog()
+        .createTable(
+            TestFixtures.TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            
PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build(),
+            null,
+            ImmutableMap.of("flink.max-continuous-empty-commits", "100000"));
+  }
+
   protected void insert(Table table, Integer id, String data) throws 
IOException {
     new GenericAppenderHelper(table, FileFormat.PARQUET, warehouseDir)
         .appendToTable(Lists.newArrayList(SimpleDataUtil.createRecord(id, 
data)));
     table.refresh();
   }
 
+  protected void update(Table table, Integer id, String oldData, String 
newData)
+      throws IOException {
+    DataFile dataFile =
+        new GenericAppenderHelper(table, FileFormat.PARQUET, warehouseDir)
+            .writeFile(Lists.newArrayList(SimpleDataUtil.createRecord(id, 
newData)));
+    DeleteFile eqDelete = writeEqualityDelete(table, id, oldData);
+
+    table.newRowDelta().addRows(dataFile).addDeletes(eqDelete).commit();
+  }
+
+  protected void update(Table table, Integer id, String oldData, String 
tempData, String newData)

Review Comment:
   this method signature is not intuitive for its purpose. maybe add a javadoc 
to explains it. like
   ```
   for the same identifier column id,  this methods simulate the following row 
operations
   - add an equality delete on oldData
   - insert tempData
   - add a position delete on tempData
   - insert newData
   ```
   



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.SizeBasedFileRewritePlanner;
+import org.apache.iceberg.flink.maintenance.operator.DataFileRewriteCommitter;
+import org.apache.iceberg.flink.maintenance.operator.DataFileRewritePlanner;
+import org.apache.iceberg.flink.maintenance.operator.DataFileRewriteRunner;
+import org.apache.iceberg.flink.maintenance.operator.TaskResultAggregator;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+/**
+ * Creates the data file rewriter data stream. Which runs a single iteration 
of the task for every
+ * {@link Trigger} event.
+ */
+public class RewriteDataFiles {
+  static final String PLANNER_TASK_NAME = "RDF Planner";
+  static final String REWRITE_TASK_NAME = "Rewrite";
+  static final String COMMIT_TASK_NAME = "Rewrite commit";
+  static final String AGGREGATOR_TASK_NAME = "Rewrite aggregator";
+
+  private RewriteDataFiles() {}
+
+  /** Creates the builder for a stream which rewrites data files for the 
table. */
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static class Builder extends 
MaintenanceTaskBuilder<RewriteDataFiles.Builder> {
+    private boolean partialProgressEnabled = false;
+    private int partialProgressMaxCommits = 10;
+    private final Map<String, String> rewriteOptions = 
Maps.newHashMapWithExpectedSize(6);
+    private long maxRewriteBytes = Long.MAX_VALUE;
+
+    /**
+     * Allows committing compacted data files in batches. For more details 
description see {@link

Review Comment:
   `For more details description see ...` doesn't read correctly. I guess you 
meant `For more detailed description, see ...`. A shorter version could be `See 
... for more details`.
   
   this comment applies to similar javadoc from other methods in this class.



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.operator;
+
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.FileRewritePlan;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+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.math.IntMath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Plans the rewrite groups using the {@link BinPackRewriteFilePlanner}. The 
input is the {@link
+ * Trigger}, the output is zero, or some {@link PlannedGroup}s.
+ */
+@Internal
+public class DataFileRewritePlanner
+    extends ProcessFunction<Trigger, DataFileRewritePlanner.PlannedGroup> {
+  static final String MESSAGE_PREFIX = "[For table {} with {}[{}] at {}]: ";
+  static final String MESSAGE_FORMAT_PREFIX = "[For table %s with {%s}[{%d}] 
at {%d}]: ";
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewritePlanner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+  private final int partialProgressMaxCommits;
+  private final long maxRewriteBytes;
+  private final Map<String, String> rewriterOptions;
+  private transient Counter errorCounter;
+
+  public DataFileRewritePlanner(
+      String tableName,
+      String taskName,
+      int taskIndex,
+      TableLoader tableLoader,
+      int newPartialProgressMaxCommits,
+      long maxRewriteBytes,
+      Map<String, String> rewriterOptions) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(rewriterOptions, "Options map should no be 
null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+    this.partialProgressMaxCommits = newPartialProgressMaxCommits;
+    this.maxRewriteBytes = maxRewriteBytes;
+    this.rewriterOptions = rewriterOptions;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    tableLoader.open();
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+  }
+
+  @Override
+  public void processElement(Trigger value, Context ctx, 
Collector<PlannedGroup> out)
+      throws Exception {
+    LOG.debug(
+        DataFileRewritePlanner.MESSAGE_PREFIX + "Creating rewrite plan",
+        tableName,
+        taskName,
+        taskIndex,
+        ctx.timestamp());
+    try {
+      SerializableTable table =
+          (SerializableTable) 
SerializableTable.copyOf(tableLoader.loadTable());
+      if (table.currentSnapshot() == null) {
+        LOG.info(
+            DataFileRewritePlanner.MESSAGE_PREFIX + "Nothing to plan for in an 
empty table",
+            tableName,
+            taskName,
+            taskIndex,
+            ctx.timestamp());
+        return;
+      }
+
+      BinPackRewriteFilePlanner planner = new BinPackRewriteFilePlanner(table);
+      planner.init(rewriterOptions);
+
+      FileRewritePlan<RewriteDataFiles.FileGroupInfo, FileScanTask, DataFile, 
RewriteFileGroup>
+          plan = planner.plan();
+
+      long rewriteBytes = 0;
+      int totalGroupCount = 0;
+      List<RewriteFileGroup> groups = 
Lists.newArrayList(plan.groups().iterator());

Review Comment:
   nit: why not just have `groups` holding the retained groups and iterating on 
the original `plan.groups().iterator()` directly? this way we don't need to do 
`groupIterator.remove()`.



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java:
##########
@@ -300,8 +300,7 @@ private DataStream<TableChange> changeStream(String 
tableName, TableLoader loade
     }
 
     private static String nameFor(MaintenanceTaskBuilder<?> streamBuilder, int 
taskIndex) {
-      return String.format(
-          "%s [%s]", streamBuilder.getClass().getSimpleName(), 
String.valueOf(taskIndex));
+      return String.format("%s [%s]", 
streamBuilder.getClass().getSimpleName(), taskIndex);

Review Comment:
   should the 2nd arg be `%d`?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.operator;
+
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.FileRewritePlan;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+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.math.IntMath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Plans the rewrite groups using the {@link BinPackRewriteFilePlanner}. The 
input is the {@link
+ * Trigger}, the output is zero, or some {@link PlannedGroup}s.
+ */
+@Internal
+public class DataFileRewritePlanner
+    extends ProcessFunction<Trigger, DataFileRewritePlanner.PlannedGroup> {
+  static final String MESSAGE_PREFIX = "[For table {} with {}[{}] at {}]: ";
+  static final String MESSAGE_FORMAT_PREFIX = "[For table %s with {%s}[{%d}] 
at {%d}]: ";
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewritePlanner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+  private final int partialProgressMaxCommits;
+  private final long maxRewriteBytes;
+  private final Map<String, String> rewriterOptions;
+  private transient Counter errorCounter;
+
+  public DataFileRewritePlanner(
+      String tableName,
+      String taskName,
+      int taskIndex,
+      TableLoader tableLoader,
+      int newPartialProgressMaxCommits,
+      long maxRewriteBytes,
+      Map<String, String> rewriterOptions) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(rewriterOptions, "Options map should no be 
null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+    this.partialProgressMaxCommits = newPartialProgressMaxCommits;
+    this.maxRewriteBytes = maxRewriteBytes;
+    this.rewriterOptions = rewriterOptions;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    tableLoader.open();
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+  }
+
+  @Override
+  public void processElement(Trigger value, Context ctx, 
Collector<PlannedGroup> out)
+      throws Exception {
+    LOG.debug(
+        DataFileRewritePlanner.MESSAGE_PREFIX + "Creating rewrite plan",
+        tableName,
+        taskName,
+        taskIndex,
+        ctx.timestamp());
+    try {
+      SerializableTable table =
+          (SerializableTable) 
SerializableTable.copyOf(tableLoader.loadTable());
+      if (table.currentSnapshot() == null) {
+        LOG.info(
+            DataFileRewritePlanner.MESSAGE_PREFIX + "Nothing to plan for in an 
empty table",
+            tableName,
+            taskName,
+            taskIndex,
+            ctx.timestamp());
+        return;
+      }
+
+      BinPackRewriteFilePlanner planner = new BinPackRewriteFilePlanner(table);
+      planner.init(rewriterOptions);
+
+      FileRewritePlan<RewriteDataFiles.FileGroupInfo, FileScanTask, DataFile, 
RewriteFileGroup>
+          plan = planner.plan();
+
+      long rewriteBytes = 0;
+      int totalGroupCount = 0;
+      List<RewriteFileGroup> groups = 
Lists.newArrayList(plan.groups().iterator());
+      ListIterator<RewriteFileGroup> groupIterator = groups.listIterator();
+      while (groupIterator.hasNext()) {
+        RewriteFileGroup group = groupIterator.next();
+        if (rewriteBytes + group.inputFilesSizeInBytes() > maxRewriteBytes) {
+          // Keep going, maybe some other group might fit in
+          LOG.info(
+              DataFileRewritePlanner.MESSAGE_PREFIX
+                  + "Skipping group {} as max rewrite size reached",
+              tableName,
+              taskName,
+              taskIndex,
+              ctx.timestamp(),
+              group);
+          groupIterator.remove();
+        } else {
+          rewriteBytes += group.inputFilesSizeInBytes();
+          ++totalGroupCount;
+        }
+      }
+
+      int groupsPerCommit =
+          IntMath.divide(totalGroupCount, partialProgressMaxCommits, 
RoundingMode.CEILING);
+
+      LOG.info(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Rewrite plan created {}",
+          tableName,
+          taskName,
+          taskIndex,
+          ctx.timestamp(),
+          groups);
+
+      for (RewriteFileGroup group : groups) {
+        LOG.debug(
+            DataFileRewritePlanner.MESSAGE_PREFIX + "Emitting {}",
+            tableName,
+            taskName,
+            taskIndex,
+            ctx.timestamp(),
+            group);
+        out.collect(new PlannedGroup(table, groupsPerCommit, group));
+      }
+    } catch (Exception e) {
+      LOG.info(

Review Comment:
   error or warn level?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.SizeBasedFileRewritePlanner;
+import org.apache.iceberg.flink.maintenance.operator.DataFileRewriteCommitter;
+import org.apache.iceberg.flink.maintenance.operator.DataFileRewritePlanner;
+import org.apache.iceberg.flink.maintenance.operator.DataFileRewriteRunner;
+import org.apache.iceberg.flink.maintenance.operator.TaskResultAggregator;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+/**
+ * Creates the data file rewriter data stream. Which runs a single iteration 
of the task for every
+ * {@link Trigger} event.
+ */
+public class RewriteDataFiles {
+  static final String PLANNER_TASK_NAME = "RDF Planner";
+  static final String REWRITE_TASK_NAME = "Rewrite";
+  static final String COMMIT_TASK_NAME = "Rewrite commit";
+  static final String AGGREGATOR_TASK_NAME = "Rewrite aggregator";
+
+  private RewriteDataFiles() {}
+
+  /** Creates the builder for a stream which rewrites data files for the 
table. */
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static class Builder extends 
MaintenanceTaskBuilder<RewriteDataFiles.Builder> {
+    private boolean partialProgressEnabled = false;
+    private int partialProgressMaxCommits = 10;
+    private final Map<String, String> rewriteOptions = 
Maps.newHashMapWithExpectedSize(6);
+    private long maxRewriteBytes = Long.MAX_VALUE;
+
+    /**
+     * Allows committing compacted data files in batches. For more details 
description see {@link
+     * org.apache.iceberg.actions.RewriteDataFiles#PARTIAL_PROGRESS_ENABLED}.
+     *
+     * @param newPartialProgressEnabled to enable partial commits
+     */
+    public Builder partialProgressEnabled(boolean newPartialProgressEnabled) {
+      this.partialProgressEnabled = newPartialProgressEnabled;
+      return this;
+    }
+
+    /**
+     * Configures the size of batches if {@link #partialProgressEnabled}. For 
more details
+     * description see {@link
+     * 
org.apache.iceberg.actions.RewriteDataFiles#PARTIAL_PROGRESS_MAX_COMMITS}.
+     *
+     * @param newPartialProgressMaxCommits to target number of the commits per 
run
+     */
+    public Builder partialProgressMaxCommits(int newPartialProgressMaxCommits) 
{
+      this.partialProgressMaxCommits = newPartialProgressMaxCommits;
+      return this;
+    }
+
+    /**
+     * Configures the maximum byte size of the rewrites for one scheduled 
compaction. This could be
+     * used to limit the resources used by the compaction.
+     *
+     * @param newMaxRewriteBytes to limit the size of the rewrites
+     */
+    public Builder maxRewriteBytes(long newMaxRewriteBytes) {
+      this.maxRewriteBytes = newMaxRewriteBytes;
+      return this;
+    }
+
+    /**
+     * Configures the target file size. For more details description see {@link
+     * org.apache.iceberg.actions.RewriteDataFiles#TARGET_FILE_SIZE_BYTES}.
+     *
+     * @param targetFileSizeBytes target file size
+     */
+    public Builder targetFileSizeBytes(long targetFileSizeBytes) {
+      this.rewriteOptions.put(
+          SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, 
String.valueOf(targetFileSizeBytes));
+      return this;
+    }
+
+    /**
+     * Configures the min file size considered for rewriting. For more details 
description see
+     * {@link SizeBasedFileRewritePlanner#MIN_FILE_SIZE_BYTES}.
+     *
+     * @param minFileSizeBytes min file size
+     */
+    public Builder minFileSizeBytes(long minFileSizeBytes) {
+      this.rewriteOptions.put(
+          SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, 
String.valueOf(minFileSizeBytes));
+      return this;
+    }
+
+    /**
+     * Configures the max file size considered for rewriting. For more details 
description see
+     * {@link SizeBasedFileRewritePlanner#MAX_FILE_SIZE_BYTES}.
+     *
+     * @param maxFileSizeBytes max file size
+     */
+    public Builder maxFileSizeBytes(long maxFileSizeBytes) {
+      this.rewriteOptions.put(
+          SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, 
String.valueOf(maxFileSizeBytes));
+      return this;
+    }
+
+    /**
+     * Configures the minimum file number after a rewrite is always initiated. 
For more details
+     * description see {@link SizeBasedFileRewritePlanner#MIN_INPUT_FILES}.
+     *
+     * @param minInputFiles min file number
+     */
+    public Builder minInputFiles(int minInputFiles) {
+      this.rewriteOptions.put(
+          SizeBasedFileRewritePlanner.MIN_INPUT_FILES, 
String.valueOf(minInputFiles));
+      return this;
+    }
+
+    /**
+     * Configures the minimum delete file number for a file after a rewrite is 
always initiated. For
+     * more details description see {@link 
BinPackRewriteFilePlanner#DELETE_FILE_THRESHOLD}.
+     *
+     * @param deleteFileThreshold min delete file number
+     */
+    public Builder deleteFileThreshold(int deleteFileThreshold) {
+      this.rewriteOptions.put(
+          BinPackRewriteFilePlanner.DELETE_FILE_THRESHOLD, 
String.valueOf(deleteFileThreshold));
+      return this;
+    }
+
+    /**
+     * Every other option is overridden, and all the files are rewritten.
+     *
+     * @param rewriteAll enables a full rewrite
+     */
+    public Builder rewriteAll(boolean rewriteAll) {
+      this.rewriteOptions.put(SizeBasedFileRewritePlanner.REWRITE_ALL, 
String.valueOf(rewriteAll));
+      return this;
+    }
+
+    /**
+     * Configures the group size for rewriting. For more details description 
see {@link
+     * SizeBasedFileRewritePlanner#MAX_FILE_GROUP_SIZE_BYTES}.
+     *
+     * @param maxFileGroupSizeBytes file group size for rewrite
+     */
+    public Builder maxFileGroupSizeBytes(long maxFileGroupSizeBytes) {
+      this.rewriteOptions.put(
+          SizeBasedFileRewritePlanner.MAX_FILE_GROUP_SIZE_BYTES,
+          String.valueOf(maxFileGroupSizeBytes));
+      return this;
+    }
+
+    /**
+     * The input is a {@link DataStream} with {@link Trigger} events and every 
event should be
+     * immediately followed by a {@link 
org.apache.flink.streaming.api.watermark.Watermark} with the

Review Comment:
   nit: import `Watermark`?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.SizeBasedFileRewritePlanner;
+import org.apache.iceberg.flink.maintenance.operator.DataFileRewriteCommitter;
+import org.apache.iceberg.flink.maintenance.operator.DataFileRewritePlanner;
+import org.apache.iceberg.flink.maintenance.operator.DataFileRewriteRunner;
+import org.apache.iceberg.flink.maintenance.operator.TaskResultAggregator;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+/**
+ * Creates the data file rewriter data stream. Which runs a single iteration 
of the task for every
+ * {@link Trigger} event.
+ */
+public class RewriteDataFiles {
+  static final String PLANNER_TASK_NAME = "RDF Planner";
+  static final String REWRITE_TASK_NAME = "Rewrite";
+  static final String COMMIT_TASK_NAME = "Rewrite commit";
+  static final String AGGREGATOR_TASK_NAME = "Rewrite aggregator";
+
+  private RewriteDataFiles() {}
+
+  /** Creates the builder for a stream which rewrites data files for the 
table. */
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static class Builder extends 
MaintenanceTaskBuilder<RewriteDataFiles.Builder> {
+    private boolean partialProgressEnabled = false;
+    private int partialProgressMaxCommits = 10;
+    private final Map<String, String> rewriteOptions = 
Maps.newHashMapWithExpectedSize(6);
+    private long maxRewriteBytes = Long.MAX_VALUE;
+
+    /**
+     * Allows committing compacted data files in batches. For more details 
description see {@link
+     * org.apache.iceberg.actions.RewriteDataFiles#PARTIAL_PROGRESS_ENABLED}.
+     *
+     * @param newPartialProgressEnabled to enable partial commits
+     */
+    public Builder partialProgressEnabled(boolean newPartialProgressEnabled) {
+      this.partialProgressEnabled = newPartialProgressEnabled;
+      return this;
+    }
+
+    /**
+     * Configures the size of batches if {@link #partialProgressEnabled}. For 
more details
+     * description see {@link
+     * 
org.apache.iceberg.actions.RewriteDataFiles#PARTIAL_PROGRESS_MAX_COMMITS}.
+     *
+     * @param newPartialProgressMaxCommits to target number of the commits per 
run
+     */
+    public Builder partialProgressMaxCommits(int newPartialProgressMaxCommits) 
{
+      this.partialProgressMaxCommits = newPartialProgressMaxCommits;
+      return this;
+    }
+
+    /**
+     * Configures the maximum byte size of the rewrites for one scheduled 
compaction. This could be
+     * used to limit the resources used by the compaction.
+     *
+     * @param newMaxRewriteBytes to limit the size of the rewrites
+     */
+    public Builder maxRewriteBytes(long newMaxRewriteBytes) {
+      this.maxRewriteBytes = newMaxRewriteBytes;
+      return this;
+    }
+
+    /**
+     * Configures the target file size. For more details description see {@link
+     * org.apache.iceberg.actions.RewriteDataFiles#TARGET_FILE_SIZE_BYTES}.
+     *
+     * @param targetFileSizeBytes target file size
+     */
+    public Builder targetFileSizeBytes(long targetFileSizeBytes) {
+      this.rewriteOptions.put(
+          SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, 
String.valueOf(targetFileSizeBytes));
+      return this;
+    }
+
+    /**
+     * Configures the min file size considered for rewriting. For more details 
description see
+     * {@link SizeBasedFileRewritePlanner#MIN_FILE_SIZE_BYTES}.
+     *
+     * @param minFileSizeBytes min file size
+     */
+    public Builder minFileSizeBytes(long minFileSizeBytes) {
+      this.rewriteOptions.put(
+          SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, 
String.valueOf(minFileSizeBytes));
+      return this;
+    }
+
+    /**
+     * Configures the max file size considered for rewriting. For more details 
description see
+     * {@link SizeBasedFileRewritePlanner#MAX_FILE_SIZE_BYTES}.
+     *
+     * @param maxFileSizeBytes max file size
+     */
+    public Builder maxFileSizeBytes(long maxFileSizeBytes) {
+      this.rewriteOptions.put(
+          SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, 
String.valueOf(maxFileSizeBytes));
+      return this;
+    }
+
+    /**
+     * Configures the minimum file number after a rewrite is always initiated. 
For more details
+     * description see {@link SizeBasedFileRewritePlanner#MIN_INPUT_FILES}.
+     *
+     * @param minInputFiles min file number
+     */
+    public Builder minInputFiles(int minInputFiles) {
+      this.rewriteOptions.put(
+          SizeBasedFileRewritePlanner.MIN_INPUT_FILES, 
String.valueOf(minInputFiles));
+      return this;
+    }
+
+    /**
+     * Configures the minimum delete file number for a file after a rewrite is 
always initiated. For
+     * more details description see {@link 
BinPackRewriteFilePlanner#DELETE_FILE_THRESHOLD}.
+     *
+     * @param deleteFileThreshold min delete file number
+     */
+    public Builder deleteFileThreshold(int deleteFileThreshold) {
+      this.rewriteOptions.put(
+          BinPackRewriteFilePlanner.DELETE_FILE_THRESHOLD, 
String.valueOf(deleteFileThreshold));
+      return this;
+    }
+
+    /**
+     * Every other option is overridden, and all the files are rewritten.

Review Comment:
   nit: the Javadoc from `SizeBaseFileRewriter` seems a little better wording.
   
   ```
    /** Overrides other options and forces rewriting of all provided files. */
   ```



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.operator;
+
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.FileRewritePlan;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+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.math.IntMath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Plans the rewrite groups using the {@link BinPackRewriteFilePlanner}. The 
input is the {@link
+ * Trigger}, the output is zero, or some {@link PlannedGroup}s.
+ */
+@Internal
+public class DataFileRewritePlanner
+    extends ProcessFunction<Trigger, DataFileRewritePlanner.PlannedGroup> {
+  static final String MESSAGE_PREFIX = "[For table {} with {}[{}] at {}]: ";
+  static final String MESSAGE_FORMAT_PREFIX = "[For table %s with {%s}[{%d}] 
at {%d}]: ";
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewritePlanner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+  private final int partialProgressMaxCommits;
+  private final long maxRewriteBytes;
+  private final Map<String, String> rewriterOptions;
+  private transient Counter errorCounter;
+
+  public DataFileRewritePlanner(
+      String tableName,
+      String taskName,
+      int taskIndex,
+      TableLoader tableLoader,
+      int newPartialProgressMaxCommits,
+      long maxRewriteBytes,
+      Map<String, String> rewriterOptions) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(rewriterOptions, "Options map should no be 
null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+    this.partialProgressMaxCommits = newPartialProgressMaxCommits;
+    this.maxRewriteBytes = maxRewriteBytes;
+    this.rewriterOptions = rewriterOptions;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    tableLoader.open();
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+  }
+
+  @Override
+  public void processElement(Trigger value, Context ctx, 
Collector<PlannedGroup> out)
+      throws Exception {
+    LOG.debug(
+        DataFileRewritePlanner.MESSAGE_PREFIX + "Creating rewrite plan",
+        tableName,
+        taskName,
+        taskIndex,
+        ctx.timestamp());
+    try {
+      SerializableTable table =
+          (SerializableTable) 
SerializableTable.copyOf(tableLoader.loadTable());
+      if (table.currentSnapshot() == null) {
+        LOG.info(

Review Comment:
   nit: wondering if it is easier to have a utility/helper method than the 
constant of `MESSAGE_PREFIX`. e.g.
   ```
   private void log(String msgSuffix, String tableName, String taskName, int 
taskIndex, long ts)
   ```



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.operator;
+
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.FileRewritePlan;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+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.math.IntMath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Plans the rewrite groups using the {@link BinPackRewriteFilePlanner}. The 
input is the {@link
+ * Trigger}, the output is zero, or some {@link PlannedGroup}s.
+ */
+@Internal
+public class DataFileRewritePlanner
+    extends ProcessFunction<Trigger, DataFileRewritePlanner.PlannedGroup> {
+  static final String MESSAGE_PREFIX = "[For table {} with {}[{}] at {}]: ";
+  static final String MESSAGE_FORMAT_PREFIX = "[For table %s with {%s}[{%d}] 
at {%d}]: ";
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewritePlanner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+  private final int partialProgressMaxCommits;
+  private final long maxRewriteBytes;
+  private final Map<String, String> rewriterOptions;
+  private transient Counter errorCounter;
+
+  public DataFileRewritePlanner(
+      String tableName,
+      String taskName,
+      int taskIndex,
+      TableLoader tableLoader,
+      int newPartialProgressMaxCommits,
+      long maxRewriteBytes,
+      Map<String, String> rewriterOptions) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(rewriterOptions, "Options map should no be 
null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+    this.partialProgressMaxCommits = newPartialProgressMaxCommits;
+    this.maxRewriteBytes = maxRewriteBytes;
+    this.rewriterOptions = rewriterOptions;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    tableLoader.open();
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+  }
+
+  @Override
+  public void processElement(Trigger value, Context ctx, 
Collector<PlannedGroup> out)
+      throws Exception {
+    LOG.debug(
+        DataFileRewritePlanner.MESSAGE_PREFIX + "Creating rewrite plan",
+        tableName,
+        taskName,
+        taskIndex,
+        ctx.timestamp());
+    try {
+      SerializableTable table =
+          (SerializableTable) 
SerializableTable.copyOf(tableLoader.loadTable());
+      if (table.currentSnapshot() == null) {
+        LOG.info(
+            DataFileRewritePlanner.MESSAGE_PREFIX + "Nothing to plan for in an 
empty table",
+            tableName,
+            taskName,
+            taskIndex,
+            ctx.timestamp());
+        return;
+      }
+
+      BinPackRewriteFilePlanner planner = new BinPackRewriteFilePlanner(table);
+      planner.init(rewriterOptions);
+
+      FileRewritePlan<RewriteDataFiles.FileGroupInfo, FileScanTask, DataFile, 
RewriteFileGroup>
+          plan = planner.plan();
+
+      long rewriteBytes = 0;
+      int totalGroupCount = 0;
+      List<RewriteFileGroup> groups = 
Lists.newArrayList(plan.groups().iterator());
+      ListIterator<RewriteFileGroup> groupIterator = groups.listIterator();
+      while (groupIterator.hasNext()) {
+        RewriteFileGroup group = groupIterator.next();
+        if (rewriteBytes + group.inputFilesSizeInBytes() > maxRewriteBytes) {
+          // Keep going, maybe some other group might fit in
+          LOG.info(
+              DataFileRewritePlanner.MESSAGE_PREFIX
+                  + "Skipping group {} as max rewrite size reached",
+              tableName,
+              taskName,
+              taskIndex,
+              ctx.timestamp(),
+              group);
+          groupIterator.remove();
+        } else {
+          rewriteBytes += group.inputFilesSizeInBytes();
+          ++totalGroupCount;
+        }
+      }
+
+      int groupsPerCommit =
+          IntMath.divide(totalGroupCount, partialProgressMaxCommits, 
RoundingMode.CEILING);
+
+      LOG.info(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Rewrite plan created {}",
+          tableName,
+          taskName,
+          taskIndex,
+          ctx.timestamp(),
+          groups);
+
+      for (RewriteFileGroup group : groups) {
+        LOG.debug(

Review Comment:
   nit: info level might be better here. the group arg/string is only metadata 
fields and doesn't contain the list of files. so it shouldn't be too long



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.operator;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING;
+
+import java.util.Collections;
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import 
org.apache.iceberg.flink.maintenance.operator.DataFileRewritePlanner.PlannedGroup;
+import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory;
+import org.apache.iceberg.flink.sink.TaskWriterFactory;
+import org.apache.iceberg.flink.source.DataIterator;
+import org.apache.iceberg.flink.source.FileScanTaskReader;
+import org.apache.iceberg.flink.source.RowDataFileScanTaskReader;
+import org.apache.iceberg.io.TaskWriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Executes a rewrite for a single {@link PlannedGroup}. Reads the files with 
the standard {@link
+ * FileScanTaskReader}, so the delete files are considered, and writes using 
the {@link
+ * TaskWriterFactory}. The output is an {@link ExecutedGroup}.
+ */
+@Internal
+public class DataFileRewriteRunner
+    extends ProcessFunction<PlannedGroup, DataFileRewriteRunner.ExecutedGroup> 
{
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewriteRunner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+
+  private transient int subTaskId;
+  private transient int attemptId;
+  private transient Counter errorCounter;
+
+  public DataFileRewriteRunner(String tableName, String taskName, int 
taskIndex) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+  }
+
+  @Override
+  public void open(Configuration parameters) {
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+
+    this.subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask();
+    this.attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber();
+  }
+
+  @Override
+  public void processElement(PlannedGroup value, Context ctx, 
Collector<ExecutedGroup> out)
+      throws Exception {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Rewriting files {} from {}",
+          tableName,
+          taskName,
+          taskIndex,
+          ctx.timestamp(),
+          value.group().info(),
+          value.group().rewrittenFiles());
+    } else {
+      LOG.info(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Rewriting {} files from {}",
+          tableName,
+          taskName,
+          taskIndex,
+          ctx.timestamp(),
+          value.group().info(),

Review Comment:
   Is the msg accurate? maybe `Rewriting group {} with {} files`?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.operator;
+
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.FileRewritePlan;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+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.math.IntMath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Plans the rewrite groups using the {@link BinPackRewriteFilePlanner}. The 
input is the {@link
+ * Trigger}, the output is zero, or some {@link PlannedGroup}s.
+ */
+@Internal
+public class DataFileRewritePlanner
+    extends ProcessFunction<Trigger, DataFileRewritePlanner.PlannedGroup> {
+  static final String MESSAGE_PREFIX = "[For table {} with {}[{}] at {}]: ";
+  static final String MESSAGE_FORMAT_PREFIX = "[For table %s with {%s}[{%d}] 
at {%d}]: ";
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewritePlanner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+  private final int partialProgressMaxCommits;
+  private final long maxRewriteBytes;
+  private final Map<String, String> rewriterOptions;
+  private transient Counter errorCounter;
+
+  public DataFileRewritePlanner(
+      String tableName,
+      String taskName,
+      int taskIndex,
+      TableLoader tableLoader,
+      int newPartialProgressMaxCommits,
+      long maxRewriteBytes,
+      Map<String, String> rewriterOptions) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(rewriterOptions, "Options map should no be 
null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+    this.partialProgressMaxCommits = newPartialProgressMaxCommits;
+    this.maxRewriteBytes = maxRewriteBytes;
+    this.rewriterOptions = rewriterOptions;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    tableLoader.open();
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+  }
+
+  @Override
+  public void processElement(Trigger value, Context ctx, 
Collector<PlannedGroup> out)
+      throws Exception {
+    LOG.debug(
+        DataFileRewritePlanner.MESSAGE_PREFIX + "Creating rewrite plan",
+        tableName,
+        taskName,
+        taskIndex,
+        ctx.timestamp());
+    try {
+      SerializableTable table =
+          (SerializableTable) 
SerializableTable.copyOf(tableLoader.loadTable());
+      if (table.currentSnapshot() == null) {
+        LOG.info(
+            DataFileRewritePlanner.MESSAGE_PREFIX + "Nothing to plan for in an 
empty table",
+            tableName,
+            taskName,
+            taskIndex,
+            ctx.timestamp());
+        return;
+      }
+
+      BinPackRewriteFilePlanner planner = new BinPackRewriteFilePlanner(table);
+      planner.init(rewriterOptions);
+
+      FileRewritePlan<RewriteDataFiles.FileGroupInfo, FileScanTask, DataFile, 
RewriteFileGroup>
+          plan = planner.plan();
+
+      long rewriteBytes = 0;
+      int totalGroupCount = 0;
+      List<RewriteFileGroup> groups = 
Lists.newArrayList(plan.groups().iterator());
+      ListIterator<RewriteFileGroup> groupIterator = groups.listIterator();
+      while (groupIterator.hasNext()) {
+        RewriteFileGroup group = groupIterator.next();
+        if (rewriteBytes + group.inputFilesSizeInBytes() > maxRewriteBytes) {
+          // Keep going, maybe some other group might fit in
+          LOG.info(
+              DataFileRewritePlanner.MESSAGE_PREFIX
+                  + "Skipping group {} as max rewrite size reached",

Review Comment:
   nit: group string can be a little long with multiple fields. maybe put the 
`{}` arg in the end? e.g.
   ```
   Skipping group as max rewrite size reached: {}
   ```
   
   otherwise, the sentence is divided by the long group arg string.



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteCommitter.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.operator;
+
+import java.io.IOException;
+import java.util.Locale;
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Commits the rewrite changes using {@link RewriteDataFilesCommitManager}. 
The input is a {@link
+ * DataFileRewriteRunner.ExecutedGroup}. Only {@link Watermark} is emitted 
which is chained to
+ * {@link TaskResultAggregator} input 1.
+ */
+@Internal
+public class DataFileRewriteCommitter extends AbstractStreamOperator<Trigger>
+    implements OneInputStreamOperator<DataFileRewriteRunner.ExecutedGroup, 
Trigger> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewriteCommitter.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+
+  private transient Table table;
+  private transient CommitService commitService;
+  private transient int processed;
+  private transient Counter errorCounter;
+  private transient Counter addedDataFileNumCounter;
+  private transient Counter addedDataFileSizeCounter;
+  private transient Counter removedDataFileNumCounter;
+  private transient Counter removedDataFileSizeCounter;
+
+  public DataFileRewriteCommitter(
+      String tableName, String taskName, int taskIndex, TableLoader 
tableLoader) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+
+    tableLoader.open();
+    this.table = tableLoader.loadTable();
+
+    MetricGroup taskMetricGroup =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex);
+    this.errorCounter = 
taskMetricGroup.counter(TableMaintenanceMetrics.ERROR_COUNTER);
+    this.addedDataFileNumCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.ADDED_DATA_FILE_NUM_METRIC);
+    this.addedDataFileSizeCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.ADDED_DATA_FILE_SIZE_METRIC);
+    this.removedDataFileNumCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.REMOVED_DATA_FILE_NUM_METRIC);
+    this.removedDataFileSizeCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.REMOVED_DATA_FILE_SIZE_METRIC);
+
+    this.processed = 0;
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFileRewriteRunner.ExecutedGroup> 
streamRecord) {
+    DataFileRewriteRunner.ExecutedGroup executedGroup = 
streamRecord.getValue();
+    try {
+      if (commitService == null) {
+        FlinkRewriteDataFilesCommitManager commitManager =
+            new FlinkRewriteDataFilesCommitManager(
+                table, executedGroup.snapshotId(), 
streamRecord.getTimestamp());
+        this.commitService = 
commitManager.service(executedGroup.groupsPerCommit());
+        commitService.start();
+      }
+
+      commitService.offer(executedGroup.group());
+      ++processed;
+    } catch (Exception e) {
+      LOG.warn(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Exception processing {}",
+          tableName,
+          taskName,
+          taskIndex,
+          streamRecord.getTimestamp(),
+          executedGroup,
+          e);
+      output.collect(TaskResultAggregator.ERROR_STREAM, new StreamRecord<>(e));
+      errorCounter.inc();
+    }
+  }
+
+  @Override
+  public void processWatermark(Watermark mark) throws Exception {
+    try {
+      if (commitService != null) {
+        commitService.close();
+        if (processed != commitService.results().size()) {
+          throw new RuntimeException(

Review Comment:
   IllegalStateException?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteCommitter.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.operator;
+
+import java.io.IOException;
+import java.util.Locale;
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Commits the rewrite changes using {@link RewriteDataFilesCommitManager}. 
The input is a {@link
+ * DataFileRewriteRunner.ExecutedGroup}. Only {@link Watermark} is emitted 
which is chained to
+ * {@link TaskResultAggregator} input 1.
+ */
+@Internal
+public class DataFileRewriteCommitter extends AbstractStreamOperator<Trigger>
+    implements OneInputStreamOperator<DataFileRewriteRunner.ExecutedGroup, 
Trigger> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewriteCommitter.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+
+  private transient Table table;
+  private transient CommitService commitService;
+  private transient int processed;
+  private transient Counter errorCounter;
+  private transient Counter addedDataFileNumCounter;
+  private transient Counter addedDataFileSizeCounter;
+  private transient Counter removedDataFileNumCounter;
+  private transient Counter removedDataFileSizeCounter;
+
+  public DataFileRewriteCommitter(
+      String tableName, String taskName, int taskIndex, TableLoader 
tableLoader) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+
+    tableLoader.open();
+    this.table = tableLoader.loadTable();
+
+    MetricGroup taskMetricGroup =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex);
+    this.errorCounter = 
taskMetricGroup.counter(TableMaintenanceMetrics.ERROR_COUNTER);
+    this.addedDataFileNumCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.ADDED_DATA_FILE_NUM_METRIC);
+    this.addedDataFileSizeCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.ADDED_DATA_FILE_SIZE_METRIC);
+    this.removedDataFileNumCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.REMOVED_DATA_FILE_NUM_METRIC);
+    this.removedDataFileSizeCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.REMOVED_DATA_FILE_SIZE_METRIC);
+
+    this.processed = 0;
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFileRewriteRunner.ExecutedGroup> 
streamRecord) {
+    DataFileRewriteRunner.ExecutedGroup executedGroup = 
streamRecord.getValue();
+    try {
+      if (commitService == null) {
+        FlinkRewriteDataFilesCommitManager commitManager =
+            new FlinkRewriteDataFilesCommitManager(
+                table, executedGroup.snapshotId(), 
streamRecord.getTimestamp());
+        this.commitService = 
commitManager.service(executedGroup.groupsPerCommit());
+        commitService.start();
+      }
+
+      commitService.offer(executedGroup.group());
+      ++processed;
+    } catch (Exception e) {
+      LOG.warn(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Exception processing {}",
+          tableName,
+          taskName,
+          taskIndex,
+          streamRecord.getTimestamp(),
+          executedGroup,
+          e);
+      output.collect(TaskResultAggregator.ERROR_STREAM, new StreamRecord<>(e));
+      errorCounter.inc();
+    }
+  }
+
+  @Override
+  public void processWatermark(Watermark mark) throws Exception {
+    try {
+      if (commitService != null) {
+        commitService.close();
+        if (processed != commitService.results().size()) {

Review Comment:
   wondering if it is even necessary to track the `processed` counter? it seems 
only used for this assertion in case the commit service has internal bug?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.operator;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING;
+
+import java.util.Collections;
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import 
org.apache.iceberg.flink.maintenance.operator.DataFileRewritePlanner.PlannedGroup;
+import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory;
+import org.apache.iceberg.flink.sink.TaskWriterFactory;
+import org.apache.iceberg.flink.source.DataIterator;
+import org.apache.iceberg.flink.source.FileScanTaskReader;
+import org.apache.iceberg.flink.source.RowDataFileScanTaskReader;
+import org.apache.iceberg.io.TaskWriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Executes a rewrite for a single {@link PlannedGroup}. Reads the files with 
the standard {@link
+ * FileScanTaskReader}, so the delete files are considered, and writes using 
the {@link
+ * TaskWriterFactory}. The output is an {@link ExecutedGroup}.
+ */
+@Internal
+public class DataFileRewriteRunner
+    extends ProcessFunction<PlannedGroup, DataFileRewriteRunner.ExecutedGroup> 
{
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewriteRunner.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+
+  private transient int subTaskId;
+  private transient int attemptId;
+  private transient Counter errorCounter;
+
+  public DataFileRewriteRunner(String tableName, String taskName, int 
taskIndex) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+  }
+
+  @Override
+  public void open(Configuration parameters) {
+    this.errorCounter =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex)
+            .counter(TableMaintenanceMetrics.ERROR_COUNTER);
+
+    this.subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask();
+    this.attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber();
+  }
+
+  @Override
+  public void processElement(PlannedGroup value, Context ctx, 
Collector<ExecutedGroup> out)
+      throws Exception {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Rewriting files {} from {}",
+          tableName,
+          taskName,
+          taskIndex,
+          ctx.timestamp(),
+          value.group().info(),
+          value.group().rewrittenFiles());
+    } else {
+      LOG.info(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Rewriting {} files from {}",
+          tableName,
+          taskName,
+          taskIndex,
+          ctx.timestamp(),
+          value.group().info(),
+          value.group().rewrittenFiles().size());
+    }
+
+    try (TaskWriter<RowData> writer = writerFor(value)) {
+      try (DataIterator<RowData> iterator = readerFor(value)) {
+        while (iterator.hasNext()) {
+          writer.write(iterator.next());
+        }
+
+        Set<DataFile> dataFiles = Sets.newHashSet(writer.dataFiles());
+        value.group().setOutputFiles(dataFiles);
+        out.collect(
+            new ExecutedGroup(
+                value.table().currentSnapshot().snapshotId(),
+                value.groupsPerCommit(),
+                value.group()));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              DataFileRewritePlanner.MESSAGE_PREFIX + "Rewritten files {} from 
{} to {}",
+              tableName,
+              taskName,
+              taskIndex,
+              ctx.timestamp(),
+              value.group().info(),
+              value.group().rewrittenFiles(),
+              value.group().addedFiles());
+        } else {
+          LOG.info(
+              DataFileRewritePlanner.MESSAGE_PREFIX + "Rewritten {} files to 
{} files",
+              tableName,
+              taskName,
+              taskIndex,
+              ctx.timestamp(),
+              value.group().rewrittenFiles().size(),
+              value.group().addedFiles().size());
+        }
+      } catch (Exception ex) {
+        LOG.info(
+            DataFileRewritePlanner.MESSAGE_PREFIX + "Exception rewriting 
datafile group {}",
+            tableName,
+            taskName,
+            taskIndex,
+            ctx.timestamp(),
+            value.group(),
+            ex);
+        ctx.output(TaskResultAggregator.ERROR_STREAM, ex);
+        errorCounter.inc();
+        abort(writer, ctx.timestamp());
+      }
+    } catch (Exception ex) {
+      LOG.info(
+          DataFileRewritePlanner.MESSAGE_PREFIX
+              + "Exception creating compaction writer for group {}",
+          tableName,
+          taskName,
+          taskIndex,
+          ctx.timestamp(),
+          value.group(),
+          ex);
+      ctx.output(TaskResultAggregator.ERROR_STREAM, ex);
+      errorCounter.inc();
+    }
+  }
+
+  private TaskWriter<RowData> writerFor(PlannedGroup value) {
+    String formatString =
+        PropertyUtil.propertyAsString(
+            value.table().properties(),
+            TableProperties.DEFAULT_FILE_FORMAT,
+            TableProperties.DEFAULT_FILE_FORMAT_DEFAULT);
+    RowDataTaskWriterFactory factory =
+        new RowDataTaskWriterFactory(
+            value.table(),
+            FlinkSchemaUtil.convert(value.table().schema()),
+            value.group().inputSplitSize(),
+            FileFormat.fromString(formatString),
+            value.table().properties(),
+            null,
+            false);
+    factory.initialize(subTaskId, attemptId);
+    return factory.create();
+  }
+
+  private DataIterator<RowData> readerFor(PlannedGroup value) {
+    RowDataFileScanTaskReader reader =
+        new RowDataFileScanTaskReader(
+            value.table().schema(),
+            value.table().schema(),
+            PropertyUtil.propertyAsString(value.table().properties(), 
DEFAULT_NAME_MAPPING, null),
+            false,
+            Collections.emptyList());
+    return new DataIterator<>(
+        reader,
+        new BaseCombinedScanTask(value.group().fileScanTasks()),
+        value.table().io(),
+        value.table().encryption());
+  }
+
+  private void abort(TaskWriter<RowData> writer, long timestamp) {
+    try {
+      LOG.info(
+          DataFileRewritePlanner.MESSAGE_PREFIX
+              + "Aborting rewrite for (subTaskId {}, attemptId {})",
+          tableName,
+          taskName,
+          taskIndex,
+          timestamp,
+          subTaskId,
+          attemptId);
+      writer.abort();
+    } catch (Exception inner) {

Review Comment:
   nit: `inner` -> `e`



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java:
##########
@@ -37,16 +39,54 @@ class MaintenanceTaskTestBase extends OperatorTestBase {
   @RegisterExtension MaintenanceTaskInfraExtension infra = new 
MaintenanceTaskInfraExtension();
 
   void runAndWaitForSuccess(
+      StreamExecutionEnvironment env,
+      ManualSource<Trigger> triggerSource,
+      CollectingSink<TaskResult> collectingSink)
+      throws Exception {
+    runAndWaitForResult(env, triggerSource, collectingSink, false, null, () -> 
true);
+  }
+
+  void runAndWaitForSuccess(
+      StreamExecutionEnvironment env,
+      ManualSource<Trigger> triggerSource,
+      CollectingSink<TaskResult> collectingSink,
+      Supplier<Boolean> waitForCondition)
+      throws Exception {
+    runAndWaitForResult(env, triggerSource, collectingSink, false, null, 
waitForCondition);
+  }
+
+  Configuration runAndWaitForSavepoint(
+      StreamExecutionEnvironment env,
+      ManualSource<Trigger> triggerSource,
+      CollectingSink<TaskResult> collectingSink,
+      File savepointDir)
+      throws Exception {
+
+    return runAndWaitForResult(env, triggerSource, collectingSink, false, 
savepointDir, () -> true);
+  }
+
+  void runAndWaitForFailure(
+      StreamExecutionEnvironment env,
+      ManualSource<Trigger> triggerSource,
+      CollectingSink<TaskResult> collectingSink)
+      throws Exception {
+    runAndWaitForResult(env, triggerSource, collectingSink, true, null, () -> 
true);
+  }
+
+  Configuration runAndWaitForResult(

Review Comment:
   what is the return value used for?



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java:
##########
@@ -48,16 +48,7 @@ class MaintenanceTaskInfraExtension implements 
BeforeEachCallback {
 
   @Override
   public void beforeEach(ExtensionContext context) {
-    env = StreamExecutionEnvironment.getExecutionEnvironment();
-    source = new ManualSource<>(env, TypeInformation.of(Trigger.class));
-    // Adds the watermark to mimic the behaviour expected for the input of the 
maintenance tasks
-    triggerStream =
-        source
-            .dataStream()
-            .assignTimestampsAndWatermarks(new 
TableMaintenance.PunctuatedWatermarkStrategy())
-            .name(IGNORED_OPERATOR_NAME)
-            .forceNonParallel();
-    sink = new CollectingSink<>();
+    init(StreamExecutionEnvironment.getExecutionEnvironment());

Review Comment:
   why this refactoring? would `init` method be called by some other classes?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteCommitter.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.operator;
+
+import java.io.IOException;
+import java.util.Locale;
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Commits the rewrite changes using {@link RewriteDataFilesCommitManager}. 
The input is a {@link
+ * DataFileRewriteRunner.ExecutedGroup}. Only {@link Watermark} is emitted 
which is chained to
+ * {@link TaskResultAggregator} input 1.
+ */
+@Internal
+public class DataFileRewriteCommitter extends AbstractStreamOperator<Trigger>
+    implements OneInputStreamOperator<DataFileRewriteRunner.ExecutedGroup, 
Trigger> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataFileRewriteCommitter.class);
+
+  private final String tableName;
+  private final String taskName;
+  private final int taskIndex;
+  private final TableLoader tableLoader;
+
+  private transient Table table;
+  private transient CommitService commitService;
+  private transient int processed;
+  private transient Counter errorCounter;
+  private transient Counter addedDataFileNumCounter;
+  private transient Counter addedDataFileSizeCounter;
+  private transient Counter removedDataFileNumCounter;
+  private transient Counter removedDataFileSizeCounter;
+
+  public DataFileRewriteCommitter(
+      String tableName, String taskName, int taskIndex, TableLoader 
tableLoader) {
+    Preconditions.checkNotNull(tableName, "Table name should no be null");
+    Preconditions.checkNotNull(taskName, "Task name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+
+    this.tableName = tableName;
+    this.taskName = taskName;
+    this.taskIndex = taskIndex;
+    this.tableLoader = tableLoader;
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+
+    tableLoader.open();
+    this.table = tableLoader.loadTable();
+
+    MetricGroup taskMetricGroup =
+        TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, 
taskName, taskIndex);
+    this.errorCounter = 
taskMetricGroup.counter(TableMaintenanceMetrics.ERROR_COUNTER);
+    this.addedDataFileNumCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.ADDED_DATA_FILE_NUM_METRIC);
+    this.addedDataFileSizeCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.ADDED_DATA_FILE_SIZE_METRIC);
+    this.removedDataFileNumCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.REMOVED_DATA_FILE_NUM_METRIC);
+    this.removedDataFileSizeCounter =
+        
taskMetricGroup.counter(TableMaintenanceMetrics.REMOVED_DATA_FILE_SIZE_METRIC);
+
+    this.processed = 0;
+  }
+
+  @Override
+  public void processElement(StreamRecord<DataFileRewriteRunner.ExecutedGroup> 
streamRecord) {
+    DataFileRewriteRunner.ExecutedGroup executedGroup = 
streamRecord.getValue();
+    try {
+      if (commitService == null) {
+        FlinkRewriteDataFilesCommitManager commitManager =
+            new FlinkRewriteDataFilesCommitManager(
+                table, executedGroup.snapshotId(), 
streamRecord.getTimestamp());
+        this.commitService = 
commitManager.service(executedGroup.groupsPerCommit());
+        commitService.start();
+      }
+
+      commitService.offer(executedGroup.group());
+      ++processed;
+    } catch (Exception e) {
+      LOG.warn(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Exception processing {}",
+          tableName,
+          taskName,
+          taskIndex,
+          streamRecord.getTimestamp(),
+          executedGroup,
+          e);
+      output.collect(TaskResultAggregator.ERROR_STREAM, new StreamRecord<>(e));
+      errorCounter.inc();
+    }
+  }
+
+  @Override
+  public void processWatermark(Watermark mark) throws Exception {
+    try {
+      if (commitService != null) {
+        commitService.close();
+        if (processed != commitService.results().size()) {
+          throw new RuntimeException(
+              String.format(
+                  Locale.ROOT,
+                  DataFileRewritePlanner.MESSAGE_FORMAT_PREFIX
+                      + "From %d commits only %d were successful",
+                  tableName,
+                  taskName,
+                  taskIndex,
+                  mark.getTimestamp(),
+                  processed,
+                  commitService.results().size()));
+        }
+      }
+
+      LOG.info(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Successfully completed data 
file compaction",
+          tableName,
+          taskName,
+          taskIndex,
+          mark.getTimestamp());
+    } catch (Exception e) {
+      LOG.warn(
+          DataFileRewritePlanner.MESSAGE_PREFIX + "Exception closing commit 
service",
+          tableName,
+          taskName,
+          taskIndex,
+          mark.getTimestamp(),
+          e);
+      output.collect(TaskResultAggregator.ERROR_STREAM, new StreamRecord<>(e));
+      errorCounter.inc();
+    }
+
+    // Cleanup
+    this.commitService = null;
+    this.processed = 0;
+
+    super.processWatermark(mark);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (commitService != null) {
+      commitService.close();
+    }
+  }
+
+  private class FlinkRewriteDataFilesCommitManager extends 
RewriteDataFilesCommitManager {
+    private final long timestamp;
+
+    FlinkRewriteDataFilesCommitManager(Table table, long snapshotId, long 
timestamp) {

Review Comment:
   nit: `startingSnapshotId` is probably more clear



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java:
##########
@@ -0,0 +1,409 @@
+/*
+ * 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 static org.apache.iceberg.flink.SimpleDataUtil.createRecord;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.COMMIT_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.PLANNER_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.REWRITE_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ADDED_DATA_FILE_NUM_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ADDED_DATA_FILE_SIZE_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ERROR_COUNTER;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.REMOVED_DATA_FILE_NUM_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.REMOVED_DATA_FILE_SIZE_METRIC;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+import java.util.stream.StreamSupport;
+import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import 
org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.Test;
+
+class TestRewriteDataFiles extends MaintenanceTaskTestBase {
+  @Test
+  void testRewriteUnpartitioned() throws Exception {
+    Table table = createTable();
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+    insert(table, 3, "c");
+    insert(table, 4, "d");
+
+    appendRewriteDataFiles(
+        RewriteDataFiles.builder()
+            .parallelism(2)
+            .deleteFileThreshold(10)
+            .targetFileSizeBytes(1_000_000L)
+            .maxFileGroupSizeBytes(10_000_000L)
+            .maxFileSizeBytes(2_000_000L)
+            .minFileSizeBytes(500_000L)
+            .minInputFiles(2)
+            .partialProgressEnabled(true)
+            .partialProgressMaxCommits(1)
+            .maxRewriteBytes(100_000L)
+            .rewriteAll(false));
+
+    runAndWaitForSuccess(infra.env(), infra.source(), infra.sink());
+
+    assertFileNum(table, 1, 0);
+
+    SimpleDataUtil.assertTableRecords(
+        table,
+        ImmutableList.of(
+            createRecord(1, "a"),
+            createRecord(2, "b"),
+            createRecord(3, "c"),
+            createRecord(4, "d")));
+  }
+
+  @Test
+  void testRewritePartitioned() throws Exception {
+    Table table = createPartitionedTable();
+    insertPartitioned(table, 1, "p1");
+    insertPartitioned(table, 2, "p1");
+    insertPartitioned(table, 3, "p2");
+    insertPartitioned(table, 4, "p2");
+
+    appendRewriteDataFiles();
+
+    runAndWaitForSuccess(infra.env(), infra.source(), infra.sink());
+
+    assertFileNum(table, 2, 0);
+
+    SimpleDataUtil.assertTableRecords(
+        table,
+        ImmutableList.of(
+            createRecord(1, "p1"),
+            createRecord(2, "p1"),
+            createRecord(3, "p2"),
+            createRecord(4, "p2")));
+  }
+
+  @Test
+  void testFailure() throws Exception {
+    Table table = createTable();
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+
+    appendRewriteDataFiles();
+
+    runAndWaitForFailure(infra.env(), infra.source(), infra.sink());
+
+    // Check the metrics
+    MetricsReporterFactoryForTests.assertCounters(
+        new ImmutableMap.Builder<List<String>, Long>()
+            .put(
+                ImmutableList.of(
+                    PLANNER_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ERROR_COUNTER),
+                1L)
+            .put(
+                ImmutableList.of(
+                    REWRITE_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ERROR_COUNTER),
+                0L)
+            .put(
+                ImmutableList.of(
+                    COMMIT_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ERROR_COUNTER),
+                0L)
+            .put(
+                ImmutableList.of(
+                    COMMIT_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ADDED_DATA_FILE_NUM_METRIC),
+                1L)

Review Comment:
   so compaction did happen. maybe add some code comments to explain the 
expected values here.



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LogUtil.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.operator;
+
+class LogUtil {
+  static final String MESSAGE_PREFIX = "[For table {} with {}[{}] at {}]: ";
+  static final String MESSAGE_FORMAT_PREFIX = "[For table %s with {%s}[{%d}] 
at {%d}]: ";
+
+  private LogUtil() {}

Review Comment:
   I personally found log msg prefix less readable. I have to go find the 
constant definition to understand the log message. I don't mind a little 
repetition instead so that the log msg is fully contained in the log lines.



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java:
##########
@@ -0,0 +1,409 @@
+/*
+ * 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 static org.apache.iceberg.flink.SimpleDataUtil.createRecord;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.COMMIT_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.PLANNER_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.api.RewriteDataFiles.REWRITE_TASK_NAME;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ADDED_DATA_FILE_NUM_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ADDED_DATA_FILE_SIZE_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.ERROR_COUNTER;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.REMOVED_DATA_FILE_NUM_METRIC;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.REMOVED_DATA_FILE_SIZE_METRIC;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+import java.util.stream.StreamSupport;
+import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import 
org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.Test;
+
+class TestRewriteDataFiles extends MaintenanceTaskTestBase {
+  @Test
+  void testRewriteUnpartitioned() throws Exception {
+    Table table = createTable();
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+    insert(table, 3, "c");
+    insert(table, 4, "d");
+
+    appendRewriteDataFiles(
+        RewriteDataFiles.builder()
+            .parallelism(2)
+            .deleteFileThreshold(10)
+            .targetFileSizeBytes(1_000_000L)
+            .maxFileGroupSizeBytes(10_000_000L)
+            .maxFileSizeBytes(2_000_000L)
+            .minFileSizeBytes(500_000L)
+            .minInputFiles(2)
+            .partialProgressEnabled(true)
+            .partialProgressMaxCommits(1)
+            .maxRewriteBytes(100_000L)
+            .rewriteAll(false));
+
+    runAndWaitForSuccess(infra.env(), infra.source(), infra.sink());
+
+    assertFileNum(table, 1, 0);
+
+    SimpleDataUtil.assertTableRecords(
+        table,
+        ImmutableList.of(
+            createRecord(1, "a"),
+            createRecord(2, "b"),
+            createRecord(3, "c"),
+            createRecord(4, "d")));
+  }
+
+  @Test
+  void testRewritePartitioned() throws Exception {
+    Table table = createPartitionedTable();
+    insertPartitioned(table, 1, "p1");
+    insertPartitioned(table, 2, "p1");
+    insertPartitioned(table, 3, "p2");
+    insertPartitioned(table, 4, "p2");
+
+    appendRewriteDataFiles();
+
+    runAndWaitForSuccess(infra.env(), infra.source(), infra.sink());
+
+    assertFileNum(table, 2, 0);
+
+    SimpleDataUtil.assertTableRecords(
+        table,
+        ImmutableList.of(
+            createRecord(1, "p1"),
+            createRecord(2, "p1"),
+            createRecord(3, "p2"),
+            createRecord(4, "p2")));
+  }
+
+  @Test
+  void testFailure() throws Exception {
+    Table table = createTable();
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+
+    appendRewriteDataFiles();
+
+    runAndWaitForFailure(infra.env(), infra.source(), infra.sink());
+
+    // Check the metrics
+    MetricsReporterFactoryForTests.assertCounters(
+        new ImmutableMap.Builder<List<String>, Long>()
+            .put(
+                ImmutableList.of(
+                    PLANNER_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ERROR_COUNTER),
+                1L)
+            .put(
+                ImmutableList.of(
+                    REWRITE_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ERROR_COUNTER),
+                0L)
+            .put(
+                ImmutableList.of(
+                    COMMIT_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ERROR_COUNTER),
+                0L)
+            .put(
+                ImmutableList.of(
+                    COMMIT_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ADDED_DATA_FILE_NUM_METRIC),
+                1L)
+            .put(
+                ImmutableList.of(
+                    COMMIT_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    ADDED_DATA_FILE_SIZE_METRIC),
+                -1L)
+            .put(
+                ImmutableList.of(
+                    COMMIT_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    REMOVED_DATA_FILE_NUM_METRIC),
+                2L)
+            .put(
+                ImmutableList.of(
+                    COMMIT_TASK_NAME + "[0]",
+                    DUMMY_TABLE_NAME,
+                    DUMMY_TASK_NAME,
+                    "0",
+                    REMOVED_DATA_FILE_SIZE_METRIC),
+                -1L)
+            .build());
+  }
+
+  @Test
+  void testUidAndSlotSharingGroup() {
+    createTable();
+
+    RewriteDataFiles.builder()
+        .slotSharingGroup(SLOT_SHARING_GROUP)
+        .uidSuffix(UID_SUFFIX)
+        .append(
+            infra.triggerStream(),
+            DUMMY_TABLE_NAME,
+            DUMMY_TASK_NAME,
+            0,
+            tableLoader(),
+            "OTHER",
+            "OTHER",
+            1)
+        .sinkTo(infra.sink());
+
+    checkUidsAreSet(infra.env(), UID_SUFFIX);
+    checkSlotSharingGroupsAreSet(infra.env(), SLOT_SHARING_GROUP);
+  }
+
+  @Test
+  void testUidAndSlotSharingGroupUnset() {
+    createTable();
+
+    RewriteDataFiles.builder()
+        .append(
+            infra.triggerStream(),
+            DUMMY_TABLE_NAME,
+            DUMMY_TASK_NAME,
+            0,
+            tableLoader(),
+            UID_SUFFIX,
+            StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP,
+            1)
+        .sinkTo(infra.sink());
+
+    checkUidsAreSet(infra.env(), null);
+    checkSlotSharingGroupsAreSet(infra.env(), null);
+  }
+
+  @Test
+  void testMetrics() throws Exception {
+    Table table = createTable();
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+
+    appendRewriteDataFiles();
+
+    runAndWaitForSuccess(infra.env(), infra.source(), infra.sink());
+
+    // Check the metrics

Review Comment:
   duplicate lines 



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