mxm commented on code in PR #11497: URL: https://github.com/apache/iceberg/pull/11497#discussion_r1873507376
########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteExecutor.java: ########## @@ -0,0 +1,257 @@ +/* + * 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 DataFileRewriteExecutor + extends ProcessFunction<PlannedGroup, DataFileRewriteExecutor.ExecutedGroup> { + private static final Logger LOG = LoggerFactory.getLogger(DataFileRewriteExecutor.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 DataFileRewriteExecutor(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( + LogUtil.MESSAGE_PREFIX + "Rewriting files {} from {}", + tableName, + taskName, + taskIndex, + ctx.timestamp(), + value.group().info(), + value.group().rewrittenFiles()); + } else { + LOG.info( + LogUtil.MESSAGE_PREFIX + "Rewriting {} files", + tableName, + taskName, + taskIndex, + ctx.timestamp(), + 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( + LogUtil.MESSAGE_PREFIX + "Rewritten files {} from {} to {}", + tableName, + taskName, + taskIndex, + ctx.timestamp(), + value.group().info(), + value.group().rewrittenFiles(), + value.group().addedFiles()); + } else { + LOG.info( + LogUtil.MESSAGE_PREFIX + "Rewritten {} files to {} files", + tableName, + taskName, + taskIndex, + ctx.timestamp(), + value.group().rewrittenFiles().size(), + value.group().addedFiles().size()); + } + } catch (Exception ex) { + LOG.info( + LogUtil.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( + LogUtil.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.splitSize(), + FileFormat.fromString(formatString), + value.table().properties(), + null, + false); + factory.initialize(subTaskId, attemptId); Review Comment: We could always make this configurable, but I would suggest to default to the dynamic variant. The reason mainly being that it doesn't hurt to use the up-to-date table metadata with the current Iceberg sink. Plus, it enables to run in standalone mode, where we definitely want to refresh the table metadata. Note that being static with Flink is much worse than with Spark, because of the long-running nature of Flink streaming applications. Once we attach the compaction to the Dynamic Sink, not being dynamic is out of question. -- 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