rkhachatryan commented on a change in pull request #16575: URL: https://github.com/apache/flink/pull/16575#discussion_r679004234
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/track/BackendStateRegistry.java ########## @@ -0,0 +1,298 @@ +/* + * 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.flink.runtime.state.track; + +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.runtime.state.track.TaskStateRegistryImpl.StateObjectIDExtractor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A per-StateBackend StateRegistry. Tracks the usage of {@link StateEntry} by backend and its + * snapshots. {@link StateEntry} will discard its state once all such registries are not using it. + * + * <p>Explicitly aware of checkpointing so that tracking of associations between checkpoints and + * snapshots can be reused (instead of implementing in each backend). + * + * <h1>Tracking lifecycle</h1> + * + * <ol> + * <li>Upon {@link #stateUsed(Collection) registration}, {@link StateEntry} is added to {@link + * #inActiveUse} + * <li>Once {@link #stateNotUsed(Set) not actively used} (usually after materialization), it is + * moved to {@link #inUseBySavepoints} (if used by savepoints; otherwise this step is + * skipped). At this point, <strong>any pending checkpoint or savepoint is considered as + * potentially using the state</strong> + * <li>If any savepoint is {@link #snapshotSentForSavepoint(long, Set) reported} to actually use + * it (before being aborted) then tracking is stopped (it is removed from {@link + * #inUseBySavepoints}), discard is not possible. + * <li>Once no savepoints are using it (as a result of abortion) the entry is moved to {@link + * #inUseByCheckpoints} under the highest checkpoint ID it might be used in (if no checkpoints + * are using it then this step is skipped). + * <li>Once all checkpoint potentially using it are {@link #checkpointSubsumed(long) subsumed}, + * tracking stops (i.e. it is removed from {@link #inUseByCheckpoints}). + * </ol> + * + * On each step, the entry is notified about the event. When tracking stops it might decide to + * discard the state. + * + * <h1>Checkpoint lifecycle</h1> + * + * Upon {@link #checkpointStarting(long, boolean) start}, Every checkpoint or savepoint first + * updates {@link #lastStartedCheckpoint} and is added {@link #pendingCheckpoints} or {@link + * #pendingSavepoints}. + * + * <h1>Thread safety</h1> + * + * The class is not thread-safe. Furthermore, it uses {@link StateEntry} potentially shared with + * other instances used by other threads - so external synchronization required. + * + * @param <K> type of state identifier + */ +@NotThreadSafe +class BackendStateRegistry<K> { + private final Logger LOG = LoggerFactory.getLogger(BackendStateRegistry.class); + + private final String backendId; + private final StateObjectIDExtractor<K> keyExtractor; + + private long lastStartedCheckpoint = -1L; + private final NavigableSet<Long> pendingCheckpoints = new TreeSet<>(); + private final Set<Long> pendingSavepoints = new HashSet<>(); + + private long lastSnapshottedCheckpoint = -1L; + @Nullable private Set<K> lastSnapshot = null; + + private final Map<K, StateEntry<K>> inActiveUse = new HashMap<>(); + private final Map<K, StateEntry<K>> inUseBySavepoints = new HashMap<>(); + private final NavigableMap<Long, List<StateEntry<K>>> inUseByCheckpoints = new TreeMap<>(); + + public BackendStateRegistry(String backendId, StateObjectIDExtractor<K> keyExtractor) { + this.keyExtractor = keyExtractor; + this.backendId = backendId; + } + + public void stateUsed(Collection<StateEntry<K>> entries) { + LOG.debug("State used, backend: {}, state: {}", backendId, entries); + entries.forEach(e -> inActiveUse.put(e.getKey(), e)); + } + + public void stateNotUsed(Set<K> stateIDs) { + for (K key : stateIDs) { + StateEntry<K> entry = inActiveUse.remove(key); + if (entry != null) { + entry.notActivelyUsed( + backendId, + new TreeSet<>(pendingSavepoints), + new TreeSet<>(pendingCheckpoints)); + trackNotActivelyUsedEntry(entry); + } + } + } + + private void trackNotActivelyUsedEntry(StateEntry<K> entry) { + if (entry.isUsedInSavepoints(backendId)) { + LOG.debug( + "State entry not used but {} pending savepoints exist, backend: {}, state: {}", + pendingSavepoints.size(), + backendId, + entry); + inUseBySavepoints.put(entry.getKey(), entry); + } else if (trackIfUsedByCheckpoints(entry)) { + LOG.debug( + "State entry not used but {} pending checkpoints exist, backend: {}", + pendingCheckpoints.size(), + backendId); + } else { + LOG.debug("State entry not used, backend: {}, state: {}", backendId, entry); + } + } + + public void checkpointStarting(long checkpointId, boolean isSavepoint) { + LOG.debug( + "Checkpoint started, backend: {}, checkpoint: {}, isSavepoint: {}", + backendId, + checkpointId, + isSavepoint); + Set<Long> ids = isSavepoint ? pendingSavepoints : pendingCheckpoints; + checkState( + lastStartedCheckpoint < checkpointId, + "Out of order checkpoint: %s, backend: %s, previous: %s, isSavepoint: %s", + checkpointId, + backendId, + lastStartedCheckpoint, + isSavepoint); + ids.add(checkpointId); + lastStartedCheckpoint = checkpointId; + } + + public void snapshotTaken( + long checkpointId, StateObject state, boolean inferUnusedFromPrevious) { + Set<K> newStateKeys = keyExtractor.apply(state).keySet(); + LOG.debug( + "Checkpoint performed, backend: {}, checkpoint: {}, inferUnusedFromPrevious: {}, state objects: {}", + backendId, + checkpointId, + inferUnusedFromPrevious, + newStateKeys); + if (pendingSavepoints.remove(checkpointId)) { + snapshotSentForSavepoint(checkpointId, newStateKeys); + } else if (pendingCheckpoints.contains(checkpointId)) { + // note that we don't remove this checkpoint from pending for now - wait + // until subsumed + snapshotSentForCheckpoint(checkpointId, inferUnusedFromPrevious, newStateKeys); + } else { + throw new IllegalStateException( + String.format("Unknown checkpoint %d for backend %s", checkpointId, backendId)); + } + } + + private void snapshotSentForSavepoint(long checkpointId, Set<K> newStateKeys) { Review comment: I think both checkpoints and checkpoints are currently global - tasks do not checkpoint independently of each other. The snapshot can be global or local; but in TM, it's always local (unless everything fits into a single TM :) So here it will be`local savepoint snapshot` or `local checkpoint snapshot` - and `local` seems redundant. Or did I get you wrong? -- 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]
