rkhachatryan commented on a change in pull request #16575:
URL: https://github.com/apache/flink/pull/16575#discussion_r678567541



##########
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<>();

Review comment:
       I agree, will rename it.




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


Reply via email to