pvary commented on code in PR #15996: URL: https://github.com/apache/iceberg/pull/15996#discussion_r3280593040
########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/EqualityConvertDVResolver.java: ########## @@ -0,0 +1,281 @@ +/* + * 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.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.annotation.Internal; +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.FileFormat; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.DeleteWriteResult; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ContentFileUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Keyed parallel resolver that buffers {@link DVPosition}s per data-file path, then writes Puffin + * DV files directly via {@link BaseDVFileWriter}. Plan metadata arrives broadcast on input 2, so + * every parallel task sees the cycle's metadata and can validate against the main snapshot. + * + * <p>Buffered positions are transient per-task. On failure recovery, upstream replay rebuilds them. + */ +@Internal +public class EqualityConvertDVResolver extends AbstractStreamOperator<DVMergeResult> + implements TwoInputStreamOperator<DVPosition, EqualityConvertPlanResult, DVMergeResult> { + + private static final Logger LOG = LoggerFactory.getLogger(EqualityConvertDVResolver.class); + + private final String tableName; + private final String taskName; + private final TableLoader tableLoader; + private final String targetBranch; + + private transient Table table; + private transient OutputFileFactory fileFactory; + private transient DeleteLoader deleteLoader; + private transient Map<String, List<Long>> positionsByFile; + private transient EqualityConvertPlanResult planResult; + private transient boolean hasUpstreamError; + + public EqualityConvertDVResolver( + String tableName, String taskName, TableLoader tableLoader, String targetBranch) { + this.tableName = tableName; + this.taskName = taskName; + this.tableLoader = tableLoader; + this.targetBranch = targetBranch; + } + + @Override + public void open() throws Exception { + super.open(); + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + table = tableLoader.loadTable(); + int subtaskIndex = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + fileFactory = + OutputFileFactory.builderFor(table, subtaskIndex, 0L).format(FileFormat.PUFFIN).build(); + deleteLoader = new BaseDeleteLoader(deleteFile -> table.io().newInputFile(deleteFile)); + positionsByFile = Maps.newHashMap(); + } + + @Override + public void processElement1(StreamRecord<DVPosition> record) { + DVPosition pos = record.getValue(); + if (pos.isAbort()) { + hasUpstreamError = true; + } + + if (!hasUpstreamError) { + positionsByFile + .computeIfAbsent(pos.dataFilePath(), k -> Lists.newArrayList()) + .add(pos.position()); + } + } + + @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()) { + if (hasUpstreamError) { + output.collect(new StreamRecord<>(DVMergeResult.ABORT)); + } else { + try { + resolveAndWrite(); + } catch (Exception e) { + LOG.error("Error writing DVs for table {} task {}", tableName, taskName, e); + output.collect(TaskResultAggregator.ERROR_STREAM, new StreamRecord<>(e)); + output.collect(new StreamRecord<>(DVMergeResult.ABORT)); + } + } + + positionsByFile.clear(); + hasUpstreamError = false; + planResult = null; + } + + super.processWatermark(mark); + } + + private void resolveAndWrite() throws IOException { + if (positionsByFile.isEmpty()) { + return; + } + + table.refresh(); + + Snapshot mainSnapshot = table.snapshot(targetBranch); + + // Fail fast if the main branch changed since planning, to avoid writing DV files that the + // committer would reject via validateFromSnapshot. The next cycle will reindex. + if (mainSnapshot != null + && planResult.mainSnapshotId() != null + && mainSnapshot.snapshotId() != planResult.mainSnapshotId()) { + throw new IllegalStateException( + "Main branch snapshot changed since planning: expected " + + planResult.mainSnapshotId() + + " but found: " + + mainSnapshot.snapshotId()); + } + + Map<String, PartitionAndSpec> partitions = collectDataFilePartitions(mainSnapshot); + Map<String, DeleteFile> dvs = collectExistingDVs(mainSnapshot); Review Comment: Probably we need the `specId` in the `DVPosition` (we have the specId from the equality delete. This way we still need to filter out the existing deletes for the old data files, but everything else we can generate from the available data. WDYT? -- 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]
