mxm commented on code in PR #15996: URL: https://github.com/apache/iceberg/pull/15996#discussion_r3264852722
########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/EqualityConvertCommitter.java: ########## @@ -0,0 +1,340 @@ +/* + * 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.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; +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.TwoInputStreamOperator; +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.DeleteFile; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.ContentFileUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Commits data files and DVs to the main branch. Receives {@link DVMergeResult}s from parallel + * {@link EqualityConvertDVMerger} instances (input 1) and an {@link EqualityConvertPlanResult} from + * the planner (input 2). Assembles the final file lists and commits using a {@link RowDelta} + * operation once the plan result and done-timestamp watermark have both arrived. + * + * <p>Watermarks are absorbed while a cycle is active. + * + * <p>Emits a {@link Trigger} after each cycle (commit, no-op, or error) so the downstream {@link + * TaskResultAggregator} can track task completion. This is the sole source of Trigger records for + * the Aggregator. + * + * <p>The committer is intentionally stateless: {@code bufferedResults} and {@code planResult} are + * not checkpointed. On restart Flink replays the upstream pipeline (planner -> reader -> worker -> + * DV resolver/merger) from the planner's last-checkpointed position, so both fields are rebuilt + * from the replayed stream before the cycle's done-timestamp watermark fires. Idempotency at commit + * time is ensured by the {@link #COMMITTED_STAGING_SNAPSHOT_PROPERTY} check in {@code + * isAlreadyCommitted}: if the staging snapshot is already on target, the replayed commit is + * skipped. Persisting {@code bufferedResults} would duplicate what the replay already yields and + * force the committer to reconcile two sources of truth. + * + * <p>On restart the planner re-derives its position from {@link + * #COMMITTED_STAGING_SNAPSHOT_PROPERTY} on main and replays any in-process cycle. + */ +@Internal +public class EqualityConvertCommitter extends AbstractStreamOperator<Trigger> + implements TwoInputStreamOperator<DVMergeResult, EqualityConvertPlanResult, Trigger> { + + private static final Logger LOG = LoggerFactory.getLogger(EqualityConvertCommitter.class); + + static final String COMMITTED_STAGING_SNAPSHOT_PROPERTY = "equality-convert-staging-snapshot"; + + private static final String ADDED_DV_NUM_METRIC = "addedDvNum"; + private static final String COMMIT_DURATION_MS_METRIC = "commitDurationMs"; + + private final String tableName; + private final String taskName; + private final int taskIndex; + private final TableLoader tableLoader; + private final String stagingBranch; + private final String targetBranch; + + private transient Table table; + private transient List<DVMergeResult> bufferedResults; + private transient EqualityConvertPlanResult planResult; + + private transient Counter errorCounter; + private transient Counter addedDataFileNumCounter; + private transient Counter addedDataFileSizeCounter; + private transient Counter addedDvNumCounter; + private transient Counter commitDurationMsCounter; + + public EqualityConvertCommitter( + String tableName, + String taskName, + int taskIndex, + TableLoader tableLoader, + String stagingBranch, + String targetBranch) { + this.tableName = tableName; + this.taskName = taskName; + this.taskIndex = taskIndex; + this.tableLoader = tableLoader; + this.stagingBranch = stagingBranch; + this.targetBranch = targetBranch; + } + + @Override + public void open() throws Exception { + super.open(); + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.bufferedResults = Lists.newArrayList(); + + 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.addedDvNumCounter = taskMetricGroup.counter(ADDED_DV_NUM_METRIC); + this.commitDurationMsCounter = taskMetricGroup.counter(COMMIT_DURATION_MS_METRIC); + } + + @Override + public void processElement1(StreamRecord<DVMergeResult> record) { + bufferedResults.add(record.getValue()); + } + + @Override + public void processElement2(StreamRecord<EqualityConvertPlanResult> record) { + planResult = record.getValue(); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + if (planResult != null && mark.getTimestamp() >= planResult.doneTimestamp()) { + tryCommit(); + } + + // Always forward watermarks to prevent stalling downstream. + super.processWatermark(mark); + } + + @Override + public void close() throws Exception { + super.close(); + tableLoader.close(); + } + + private void tryCommit() { + try { + commitIfNeeded(); + } catch (Exception e) { + LOG.error( + "Failed to commit equality convert result for table {} task {}[{}]", + tableName, + taskName, + taskIndex, + e); + output.collect(TaskResultAggregator.ERROR_STREAM, new StreamRecord<>(e)); + errorCounter.inc(); + } + + // Emit Trigger for the Aggregator (even on error or no-op). + output.collect(new StreamRecord<>(Trigger.create(planResult.triggerTimestamp(), taskIndex))); + + bufferedResults.clear(); + planResult = null; + } + + private void commitIfNeeded() { + for (DVMergeResult result : bufferedResults) { + if (result.hasError()) { + LOG.warn( + "Skipping commit for table {} task {}[{}]: a DV merger reported an error.", + tableName, + taskName, + taskIndex); + return; Review Comment: We already send a failure messages via the error output to the TaskResultAggregator. The aggregator will properly forward those. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
