manuzhang commented on code in PR #15996: URL: https://github.com/apache/iceberg/pull/15996#discussion_r3192683045
########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/EqualityConvertWorker.java: ########## @@ -0,0 +1,162 @@ +/* + * 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.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.KeyedProcessFunction; +import org.apache.flink.util.Collector; +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>Detects main branch changes via {@link IndexCommand#mainSnapshotId()} and clears stale index + * entries when the main snapshot changes. + */ +@Internal +public class EqualityConvertWorker + extends KeyedProcessFunction<SerializedEqualityValues, 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 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; + + @Override + public void open(OpenContext context) throws Exception { + super.open(context); + mainSnapshotVersion = + getRuntimeContext().getState(new ValueStateDescriptor<>("mainSnapshotVersion", Types.LONG)); + dataRowPositions = + getRuntimeContext() + .getListState( + new ListStateDescriptor<>("filePositions", TypeInformation.of(DVPosition.class))); + pendingRows = + getRuntimeContext() + .getListState( + new ListStateDescriptor<>("pendingRows", TypeInformation.of(DVPosition.class))); + resolveTimestamp = + getRuntimeContext().getState(new ValueStateDescriptor<>("resolveTimestamp", Types.LONG)); + resolvedDeleteNumCounter = + getRuntimeContext().getMetricGroup().counter(RESOLVED_DELETE_NUM_METRIC); + indexedKeyNumCounter = getRuntimeContext().getMetricGroup().counter(INDEXED_KEY_NUM_METRIC); + } + + @Override + public void processElement(IndexCommand cmd, Context 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(); Review Comment: `ctx.timestamp()` could be null? -- 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]
