pvary commented on code in PR #15996: URL: https://github.com/apache/iceberg/pull/15996#discussion_r3362696289
########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/EqualityConvertWorker.java: ########## @@ -0,0 +1,217 @@ +/* + * 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.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.metrics.Counter; +import org.apache.flink.streaming.api.functions.co.KeyedBroadcastProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Parallel worker that maintains a shard of the primary key index. Keyed by the full serialized + * primary key ({@link SerializedEqualityValues}), each instance handles a subset of PK values. + * + * <p>ADD_DATA_ROW commands with ts ≤ resolveTimestamp go directly into the resolution index; + * commands with ts > resolveTimestamp are staged as pending rows for the next cycle. This + * handles the race where parallel Readers deliver a later-phase record before an earlier-phase + * RESOLVE_DELETE fires. + * + * <p>Stale-index protection runs on two levels, both keyed on {@link + * IndexCommand#mainSnapshotId()}. Each key tracks the main snapshot id its stored positions were + * indexed against. When either path observes a newer id, the key's positions, pending rows, and + * resolve timer are cleared: + * + * <ul> + * <li><b>Lazy (per-key)</b>: any keyed command at the top of {@link #processElement} advances the + * version and clears stale state. Required because the broadcast and keyed inputs are + * independent streams with no ordering guarantee; without it, an ADD_DATA_ROW that arrived + * before the broadcast would be wrongly evicted. + * <li><b>Eager (all keys)</b>: a CLEAR_INDEX broadcast iterates all keys on the subtask via + * {@link KeyedBroadcastProcessFunction.Context#applyToKeyedState} and clears any whose stored + * version is older than the broadcast's. Bounds state size for PKs that were removed from + * main by an external CoW commit and won't receive any keyed command next cycle. + * </ul> + */ +@Internal +public class EqualityConvertWorker + extends KeyedBroadcastProcessFunction< + SerializedEqualityValues, IndexCommand, IndexCommand, DVPosition> { + + private static final Logger LOG = LoggerFactory.getLogger(EqualityConvertWorker.class); + + private static final String RESOLVED_DELETE_NUM_METRIC = "resolvedDeleteNum"; + private static final String INDEXED_KEY_NUM_METRIC = "indexedKeyNum"; + private static final String EAGERLY_EVICTED_KEY_NUM_METRIC = "eagerlyEvictedKeyNum"; + + public static final MapStateDescriptor<Void, Void> CLEAR_BROADCAST_DESCRIPTOR = + new MapStateDescriptor<>("eq-convert-clear-broadcast", Types.VOID, Types.VOID); + + private static final ValueStateDescriptor<Long> MAIN_SNAPSHOT_VERSION_DESCRIPTOR = + new ValueStateDescriptor<>("mainSnapshotVersion", Types.LONG); + private static final ListStateDescriptor<DVPosition> DATA_ROW_POSITIONS_DESCRIPTOR = + new ListStateDescriptor<>("filePositions", TypeInformation.of(DVPosition.class)); + private static final ListStateDescriptor<DVPosition> PENDING_ROWS_DESCRIPTOR = + new ListStateDescriptor<>("pendingRows", TypeInformation.of(DVPosition.class)); + private static final ValueStateDescriptor<Long> RESOLVE_TIMESTAMP_DESCRIPTOR = + new ValueStateDescriptor<>("resolveTimestamp", Types.LONG); + + private transient ValueState<Long> mainSnapshotVersion; + private transient ListState<DVPosition> dataRowPositions; + // Contains the row positions for updating the index with staging snapshot data, after the + // equality deletes for the staging snapshot have been resolved. + private transient ListState<DVPosition> pendingRows; + // Tracks the maximum RESOLVE_DELETE timestamp seen. ADD_DATA_ROWs with ts > resolveTimestamp + // belong to a later phase and are staged; those with ts <= resolveTimestamp go to the index. + private transient ValueState<Long> resolveTimestamp; + + private transient Counter resolvedDeleteNumCounter; + private transient Counter indexedKeyNumCounter; + private transient Counter eagerlyEvictedKeyNumCounter; + + @Override + public void open(OpenContext context) throws Exception { + super.open(context); + mainSnapshotVersion = getRuntimeContext().getState(MAIN_SNAPSHOT_VERSION_DESCRIPTOR); + dataRowPositions = getRuntimeContext().getListState(DATA_ROW_POSITIONS_DESCRIPTOR); + pendingRows = getRuntimeContext().getListState(PENDING_ROWS_DESCRIPTOR); + resolveTimestamp = getRuntimeContext().getState(RESOLVE_TIMESTAMP_DESCRIPTOR); + resolvedDeleteNumCounter = + getRuntimeContext().getMetricGroup().counter(RESOLVED_DELETE_NUM_METRIC); + indexedKeyNumCounter = getRuntimeContext().getMetricGroup().counter(INDEXED_KEY_NUM_METRIC); + eagerlyEvictedKeyNumCounter = + getRuntimeContext().getMetricGroup().counter(EAGERLY_EVICTED_KEY_NUM_METRIC); + } + + @Override + public void processElement(IndexCommand cmd, ReadOnlyContext ctx, Collector<DVPosition> out) + throws Exception { + try { + Long storedVersion = mainSnapshotVersion.value(); + if (!Objects.equals(storedVersion, cmd.mainSnapshotId())) { + LOG.info( + "Main snapshot changed from {} to {}, clearing state", + storedVersion, + cmd.mainSnapshotId()); + dataRowPositions.clear(); + pendingRows.clear(); + resolveTimestamp.clear(); + mainSnapshotVersion.update(cmd.mainSnapshotId()); + } + + long ts = ctx.timestamp(); + Long resolveTs = resolveTimestamp.value(); + + if (cmd.type() == IndexCommand.Type.ADD_DATA_ROW) { + DVPosition position = + new DVPosition(cmd.filePath(), cmd.position(), cmd.specId(), cmd.partition()); + if (resolveTs != null && ts > resolveTs) { + pendingRows.add(position); + } else { + dataRowPositions.add(position); + indexedKeyNumCounter.inc(); + } + } else if (cmd.type() == IndexCommand.Type.RESOLVE_DELETE) { + if (resolveTs == null || ts > resolveTs) { + resolveTimestamp.update(ts); + } + + ctx.timerService().registerEventTimeTimer(ts); + } + } catch (Exception e) { + LOG.error("Worker failed to process command for file={}", cmd.filePath(), e); + ctx.output(TaskResultAggregator.ERROR_STREAM, e); + out.collect(DVPosition.ABORT); + } + } + + @Override + public void processBroadcastElement(IndexCommand cmd, Context ctx, Collector<DVPosition> out) { + Preconditions.checkArgument( + cmd.type() == IndexCommand.Type.CLEAR_INDEX, + "Broadcast element must be %s", + IndexCommand.Type.CLEAR_INDEX); + + final long broadcastVersion = cmd.mainSnapshotId(); + try { + ctx.applyToKeyedState( + MAIN_SNAPSHOT_VERSION_DESCRIPTOR, + (key, versionState) -> { + Long storedVersion = versionState.value(); + if (storedVersion != null && storedVersion < broadcastVersion) { Review Comment: These are `snapshotId`s? AFAIK there is no order defined between them. They are just "random", "unique" longs. -- 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]
