mxm commented on code in PR #15996:
URL: https://github.com/apache/iceberg/pull/15996#discussion_r3377908715


##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/EqualityConvertDVResolver.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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 java.util.Set;
+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.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.PartitionField;
+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.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.ManifestEvaluator;
+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.annotations.VisibleForTesting;
+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.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ContentFileUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+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>Each buffered {@link DVPosition} carries the data file's {@code specId} 
+ encoded partition,
+ * so writing DVs needs no data-manifest scan. Existing DVs are folded into 
the rewrite (V3 allows
+ * one DV per data file): delete manifests are pruned by partition summary to 
the cycle's affected
+ * partitions, then filtered to entries referencing the affected data files. 
No cross-cycle state is
+ * kept; reads are bounded by the pruned manifest set, not the table's full DV 
history.
+ *
+ * <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, FilePositions> positionsByFile;
+  private transient EqualityConvertPlanResult planResult;
+  private transient boolean hasUpstreamError;
+  private transient int manifestsRead;
+
+  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 -> new FilePositions(pos.specId(), 
pos.partition()))
+          .positions
+          .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, DeleteFile> dvs = collectExistingDVs(mainSnapshot, 
positionsByFile.keySet());
+
+    // Include staging DVs so the writer folds them in; prevents two DVs per 
data file (V3
+    // violation).
+    for (DeleteFile sd : planResult.stagingDVFiles()) {
+      if (ContentFileUtil.isDV(sd) && sd.referencedDataFile() != null) {
+        dvs.put(sd.referencedDataFile(), sd);
+      }
+    }
+
+    BaseDVFileWriter dvWriter =
+        new BaseDVFileWriter(fileFactory, path -> loadPreviousDV(path, dvs));
+    try (dvWriter) {
+      for (Map.Entry<String, FilePositions> entry : 
positionsByFile.entrySet()) {
+        String dataFilePath = entry.getKey();
+        FilePositions filePositions = entry.getValue();
+        PartitionSpec spec = table.specs().get(filePositions.specId);
+        StructLike partition = filePositions.partition(spec.partitionType());
+
+        for (long pos : filePositions.positions) {
+          dvWriter.delete(dataFilePath, pos, spec, partition);
+        }
+      }
+    }
+
+    DeleteWriteResult result = dvWriter.result();
+    LOG.info(
+        "Wrote {} DV files (rewriting {}) for {} data files in table {} task 
{}.",
+        result.deleteFiles().size(),
+        result.rewrittenDeleteFiles().size(),
+        positionsByFile.size(),
+        tableName,
+        taskName);
+
+    output.collect(
+        new StreamRecord<>(
+            new DVMergeResult(
+                Lists.newArrayList(result.deleteFiles()),
+                Lists.newArrayList(result.rewrittenDeleteFiles()))));
+  }
+
+  @Override
+  public void close() throws Exception {
+    super.close();
+    tableLoader.close();
+  }
+
+  private Map<String, DeleteFile> collectExistingDVs(
+      Snapshot mainSnapshot, Set<String> affectedPaths) {
+    manifestsRead = 0;
+    Map<String, DeleteFile> dvs = Maps.newHashMap();
+    if (mainSnapshot == null) {
+      return dvs;
+    }
+
+    // Prune delete manifests whose partition summaries cannot cover the 
cycle's affected
+    // partitions. A DV inherits its referenced data file's spec and 
partition, so partition pruning
+    // works for DV manifests.
+    Map<Integer, ManifestEvaluator> evaluators = 
partitionEvaluators(positionsByFile);
+    for (ManifestFile manifest : mainSnapshot.deleteManifests(table.io())) {
+      ManifestEvaluator evaluator = evaluators.get(manifest.partitionSpecId());
+      if (evaluator == null || !evaluator.eval(manifest)) {
+        continue;
+      }
+
+      readDVEntries(manifest, affectedPaths, dvs);
+    }
+
+    return dvs;
+  }
+
+  private Map<Integer, ManifestEvaluator> partitionEvaluators(
+      Map<String, FilePositions> positions) {
+    Map<Integer, StructLikeWrapper> templatesBySpec = Maps.newHashMap();
+    Map<Integer, Set<StructLikeWrapper>> partitionsBySpec = Maps.newHashMap();
+    for (FilePositions filePositions : positions.values()) {
+      PartitionSpec spec = table.specs().get(filePositions.specId);
+      StructLikeWrapper template =
+          templatesBySpec.computeIfAbsent(
+              filePositions.specId, id -> 
StructLikeWrapper.forType(spec.partitionType()));
+      partitionsBySpec
+          .computeIfAbsent(filePositions.specId, k -> Sets.newHashSet())
+          
.add(template.copyFor(filePositions.partition(spec.partitionType())));
+    }
+
+    Map<Integer, ManifestEvaluator> evaluators = Maps.newHashMap();
+    for (Map.Entry<Integer, Set<StructLikeWrapper>> entry : 
partitionsBySpec.entrySet()) {
+      PartitionSpec spec = table.specs().get(entry.getKey());
+      Expression filter = partitionFilter(spec, entry.getValue());
+      evaluators.put(entry.getKey(), 
ManifestEvaluator.forPartitionFilter(filter, spec, false));
+    }
+
+    return evaluators;
+  }
+
+  private static Expression partitionFilter(PartitionSpec spec, 
Set<StructLikeWrapper> partitions) {
+    List<PartitionField> fields = spec.fields();
+    Expression anyPartition = Expressions.alwaysFalse();
+    for (StructLikeWrapper wrapper : partitions) {
+      StructLike partition = wrapper.get();
+      Expression onePartition = Expressions.alwaysTrue();
+      for (int i = 0; i < fields.size(); i++) {
+        String name = fields.get(i).name();
+        Object value = partition.get(i, Object.class);
+        Expression predicate =
+            value == null ? Expressions.isNull(name) : Expressions.equal(name, 
value);
+        onePartition = Expressions.and(onePartition, predicate);
+      }
+
+      anyPartition = Expressions.or(anyPartition, onePartition);
+    }
+
+    return anyPartition;
+  }
+
+  private void readDVEntries(
+      ManifestFile manifest, Set<String> filterPaths, Map<String, DeleteFile> 
out) {
+    manifestsRead++;
+    try (ManifestReader<DeleteFile> reader =
+        ManifestFiles.readDeleteManifest(manifest, table.io(), table.specs())) 
{
+      for (DeleteFile deleteFile : reader) {
+        if (ContentFileUtil.isDV(deleteFile)
+            && deleteFile.referencedDataFile() != null
+            && filterPaths.contains(deleteFile.referencedDataFile())) {
+          out.put(deleteFile.referencedDataFile(), deleteFile);
+        }
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to read manifest: " + 
manifest.path(), e);
+    }
+  }
+
+  @VisibleForTesting
+  int manifestsReadLastCycle() {
+    return manifestsRead;
+  }
+
+  @VisibleForTesting
+  int retainedStateSize() {
+    return positionsByFile.size();
+  }
+
+  private PositionDeleteIndex loadPreviousDV(String dataFilePath, Map<String, 
DeleteFile> dvs) {
+    DeleteFile existingDV = dvs.get(dataFilePath);
+    if (existingDV == null) {
+      return null;
+    }
+
+    return deleteLoader.loadPositionDeletes(ImmutableList.of(existingDV), 
dataFilePath);
+  }
+
+  private static final class FilePositions {
+    private final int specId;
+    private final byte[] encodedPartition;
+    private final List<Long> positions = Lists.newArrayList();

Review Comment:
   Yes! The number of positions per file could be large for some scenarios. 
Added.



##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/EqualityConvertPlanResult.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.Serializable;
+import java.util.List;
+import org.apache.flink.annotation.Internal;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+/**
+ * Result of equality convert planning. Produced by {@link 
EqualityConvertPlanner} and consumed by
+ * both {@link EqualityConvertDVResolver} (for partition info and DV merge 
gating) and {@link
+ * EqualityConvertCommitter} (for data files and staging deletes to commit).
+ *
+ * <p>A no-op cycle is encoded by {@link #stagingSnapshotId()} == {@link
+ * #NO_OP_STAGING_SNAPSHOT_ID}. Use {@link #noOp(Long, long, long)} to 
construct one and {@link
+ * #noOp()} to check.
+ *
+ * @param dataFiles new staging data files committed in the cycle
+ * @param stagingDVFiles staging DVs passed through to main, used by 
DVResolver to merge with
+ *     newly-created DVs
+ * @param stagingSnapshotId staging snapshot the cycle resolved against, or 
{@link
+ *     #NO_OP_STAGING_SNAPSHOT_ID} for a no-op cycle
+ * @param mainSnapshotId main branch snapshot ID the index was resolved 
against, used for commit
+ *     validation
+ * @param triggerTimestamp original trigger timestamp, forwarded by the 
Committer to the Aggregator
+ * @param doneTimestamp timestamp after which all phase watermarks have been 
emitted; the DVResolver
+ *     should only process the result when the watermark reaches or exceeds 
this value
+ */
+@Internal
+public record EqualityConvertPlanResult(

Review Comment:
   Renamed.



##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ConvertEqualityDeletes.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.api;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.datastream.BroadcastStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableUtil;
+import org.apache.iceberg.flink.maintenance.operator.DVMergeResult;
+import org.apache.iceberg.flink.maintenance.operator.DVPosition;
+import org.apache.iceberg.flink.maintenance.operator.EqualityConvertCommitter;
+import org.apache.iceberg.flink.maintenance.operator.EqualityConvertDVResolver;
+import org.apache.iceberg.flink.maintenance.operator.EqualityConvertPlanResult;
+import org.apache.iceberg.flink.maintenance.operator.EqualityConvertPlanner;
+import org.apache.iceberg.flink.maintenance.operator.EqualityConvertReader;
+import org.apache.iceberg.flink.maintenance.operator.EqualityConvertWorker;
+import org.apache.iceberg.flink.maintenance.operator.IndexCommand;
+import org.apache.iceberg.flink.maintenance.operator.ReadCommand;
+import org.apache.iceberg.flink.maintenance.operator.SerializedEqualityValues;
+import org.apache.iceberg.flink.maintenance.operator.TaskResultAggregator;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types.NestedField;
+
+/**
+ * Creates the equality delete to DV conversion data stream. Runs a single 
iteration of the
+ * conversion for every {@link Trigger} event.
+ *
+ * <p>The pipeline reads equality delete files from a staging branch, converts 
them to deletion
+ * vectors (DVs) using a primary key index stored in Flink state, and commits 
the data files and DVs
+ * to the main branch.
+ *
+ * <p>The conversion is split into parallel stages:
+ *
+ * <ol>
+ *   <li>Planner (p=1): scans staging branch, emits file-level ReadCommands 
with phase timestamps
+ *   <li>Reader (p=N): reads files, emits row-level IndexCommands
+ *   <li>Worker (p=N): maintains PK index shards, resolves equality deletes to 
DV positions
+ *   <li>DVResolver (p=N, keyed by data file path): buffers positions per 
file, writes Puffin DVs
+ *       inline
+ *   <li>Committer (p=1): commits data files and DVs to main branch
+ * </ol>
+ *
+ * <p>Mutual exclusion with concurrent maintenance tasks (e.g. compaction) is 
enforced by the Flink
+ * maintenance framework lock.
+ */
+@Experimental
+public class ConvertEqualityDeletes {
+  static final String PLANNER_TASK_NAME = "EqConvert Planner";
+  static final String READER_TASK_NAME = "EqConvert Reader";
+  static final String WORKER_TASK_NAME = "EqConvert Worker";
+  static final String DV_RESOLVER_TASK_NAME = "EqConvert DVResolver";
+  static final String UPSTREAM_ABORT_TASK_NAME = "EqConvert UpstreamAbort";
+  static final String COMMIT_TASK_NAME = "EqConvert Commit";
+  static final String AGGREGATOR_TASK_NAME = "EqConvert Aggregator";
+
+  private ConvertEqualityDeletes() {}
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static class Builder extends MaintenanceTaskBuilder<Builder> {
+    private String stagingBranch;
+    private String targetBranch = SnapshotRef.MAIN_BRANCH;
+    private List<String> equalityFieldColumns = Collections.emptyList();
+
+    @Override
+    String maintenanceTaskName() {
+      return "ConvertEqualityDeletes";
+    }
+
+    /** Sets the staging branch name that holds the equality delete files and 
data files. */
+    public Builder stagingBranch(String newStagingBranch) {
+      this.stagingBranch = newStagingBranch;
+      return this;
+    }
+
+    /**
+     * Sets the target branch where converted data files and DVs are 
committed. Defaults to the main
+     * branch.
+     */
+    public Builder targetBranch(String newTargetBranch) {
+      this.targetBranch = newTargetBranch;
+      return this;
+    }
+
+    /**
+     * Sets the equality field columns used by the worker index. Required. 
Must match the equality
+     * field columns the writer uses for staging eq-delete files; mismatched 
eq-deletes fail the
+     * cycle. Mirrors {@link
+     * org.apache.iceberg.flink.sink.IcebergSink.Builder#equalityFieldColumns}.
+     *
+     * <p>Partition source columns of every spec on the table must be a subset 
of these columns
+     * (FlinkSink / IcebergSink contract). The converter keys data rows by 
{@code (specId, equality
+     * values)} and assumes the partition contains the equality values.
+     */
+    public Builder equalityFieldColumns(List<String> columns) {
+      Preconditions.checkNotNull(columns, "equalityFieldColumns must not be 
null");
+      Preconditions.checkArgument(!columns.isEmpty(), "equalityFieldColumns 
must not be empty");
+      this.equalityFieldColumns = 
Collections.unmodifiableList(Lists.newArrayList(columns));
+      return this;
+    }
+
+    @Override
+    DataStream<TaskResult> append(DataStream<Trigger> trigger) {
+      Preconditions.checkNotNull(stagingBranch, "stagingBranch must be set");
+      Preconditions.checkArgument(
+          !equalityFieldColumns.isEmpty(), "equalityFieldColumns must be set 
on the builder");
+      List<Integer> eqFieldIds = resolveEqualityFieldIds();
+
+      // Planner (p=1): emits ReadCommands with phase timestamps and watermarks
+      SingleOutputStreamOperator<ReadCommand> planned =
+          setSlotSharingGroup(
+              trigger
+                  .transform(
+                      operatorName(PLANNER_TASK_NAME),
+                      TypeInformation.of(ReadCommand.class),
+                      new EqualityConvertPlanner(
+                          tableName(),
+                          taskName(),
+                          tableLoader(),
+                          stagingBranch,
+                          targetBranch,
+                          eqFieldIds))
+                  .uid(PLANNER_TASK_NAME + uidSuffix())
+                  .forceNonParallel());
+
+      // Reader (p=N): reads files, emits IndexCommands
+      SingleOutputStreamOperator<IndexCommand> index =
+          setSlotSharingGroup(
+              planned
+                  .rebalance()
+                  .process(new EqualityConvertReader(tableLoader(), 
eqFieldIds))
+                  .name(operatorName(READER_TASK_NAME))
+                  .uid(READER_TASK_NAME + uidSuffix())
+                  .setParallelism(parallelism()));
+
+      // Broadcast from the planner to the worker to clear the entire index
+      BroadcastStream<IndexCommand> clearIndexBroadcast =
+          planned
+              .getSideOutput(EqualityConvertPlanner.CLEAR_BROADCAST_STREAM)
+              .broadcast(EqualityConvertWorker.CLEAR_BROADCAST_DESCRIPTOR);
+
+      // Worker (p=N): keyed by full PK, phase-aware buffering.
+      SingleOutputStreamOperator<DVPosition> dvPositions =
+          setSlotSharingGroup(
+              index
+                  .keyBy(IndexCommand::key, 
TypeInformation.of(SerializedEqualityValues.class))
+                  .connect(clearIndexBroadcast)
+                  .process(new EqualityConvertWorker())
+                  .name(operatorName(WORKER_TASK_NAME))
+                  .uid(WORKER_TASK_NAME + uidSuffix())
+                  .setParallelism(parallelism()));
+
+      // Reader-side abort signals bypass the Worker and feed the DVResolver 
directly, so a reader
+      // failure can short-circuit the cycle without waiting on a keyed 
shuffle.
+      DataStream<DVPosition> readerAborts =
+          index.getSideOutput(EqualityConvertReader.READER_ABORT_STREAM);
+      DataStream<DVPosition> dvPositionsWithAborts = 
dvPositions.union(readerAborts);
+
+      // Metadata side output from planner
+      DataStream<EqualityConvertPlanResult> metadata =
+          planned.getSideOutput(EqualityConvertPlanner.METADATA_STREAM);
+
+      // DVResolver (p=N, keyed by data file path): groups positions per file, 
writes Puffin DV
+      // files inline, emits DVMergeResult per task. Plan metadata broadcast 
so every subtask sees
+      // it.
+      SingleOutputStreamOperator<DVMergeResult> resolved =
+          setSlotSharingGroup(
+              dvPositionsWithAborts
+                  .keyBy(DVPosition::dataFilePath)
+                  .connect(metadata.broadcast())
+                  .transform(
+                      operatorName(DV_RESOLVER_TASK_NAME),
+                      TypeInformation.of(DVMergeResult.class),
+                      new EqualityConvertDVResolver(
+                          tableName(), taskName(), tableLoader(), 
targetBranch))
+                  .uid(DV_RESOLVER_TASK_NAME + uidSuffix())
+                  .setParallelism(parallelism()));
+
+      // Upstream errors become abort signals so a partial read never commits. 
The same error side
+      // outputs also feed the aggregator below to surface the exception in 
TaskResult; the two
+      // consumers serve different purposes and must both exist.
+      DataStream<DVMergeResult> upstreamAborts =
+          setSlotSharingGroup(
+              index
+                  .getSideOutput(TaskResultAggregator.ERROR_STREAM)
+                  
.union(dvPositions.getSideOutput(TaskResultAggregator.ERROR_STREAM))
+                  .map(e -> DVMergeResult.ABORT)
+                  .returns(TypeInformation.of(DVMergeResult.class))
+                  .name(operatorName(UPSTREAM_ABORT_TASK_NAME))
+                  .uid(UPSTREAM_ABORT_TASK_NAME + uidSuffix())
+                  .forceNonParallel());
+
+      // Committer (p=1): commits data files + DVs to main.
+      SingleOutputStreamOperator<Trigger> committed =
+          setSlotSharingGroup(
+              resolved
+                  .union(upstreamAborts)
+                  .connect(metadata)
+                  .transform(
+                      operatorName(COMMIT_TASK_NAME),
+                      TypeInformation.of(Trigger.class),
+                      new EqualityConvertCommitter(
+                          tableName(), taskName(), tableLoader(), 
stagingBranch, targetBranch))
+                  .uid(COMMIT_TASK_NAME + uidSuffix())
+                  .forceNonParallel());
+
+      // Aggregator (p=1): collects errors and emits TaskResult.
+      return setSlotSharingGroup(
+          committed
+              .connect(
+                  planned
+                      .getSideOutput(TaskResultAggregator.ERROR_STREAM)
+                      
.union(index.getSideOutput(TaskResultAggregator.ERROR_STREAM))
+                      
.union(dvPositions.getSideOutput(TaskResultAggregator.ERROR_STREAM))
+                      
.union(resolved.getSideOutput(TaskResultAggregator.ERROR_STREAM))
+                      
.union(committed.getSideOutput(TaskResultAggregator.ERROR_STREAM)))
+              .transform(
+                  operatorName(AGGREGATOR_TASK_NAME),
+                  TypeInformation.of(TaskResult.class),
+                  new TaskResultAggregator(tableName(), taskName(), index()))
+              .uid(AGGREGATOR_TASK_NAME + uidSuffix())
+              .forceNonParallel());
+    }
+
+    private List<Integer> resolveEqualityFieldIds() {
+      if (!tableLoader().isOpen()) {
+        tableLoader().open();
+      }
+
+      Table table = tableLoader().loadTable();
+      int formatVersion = TableUtil.formatVersion(table);
+      Preconditions.checkArgument(
+          formatVersion >= 3,
+          "ConvertEqualityDeletes requires table format version >= 3 (DVs), "
+              + "but table '%s' is version %s",
+          tableName(),
+          formatVersion);
+
+      Schema schema = table.schema();
+      List<Integer> fieldIds = 
Lists.newArrayListWithCapacity(equalityFieldColumns.size());
+      for (String column : equalityFieldColumns) {
+        NestedField field = schema.findField(column);
+        Preconditions.checkArgument(

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: [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]

Reply via email to