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


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



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



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



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