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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/track/TaskStateRegistryImpl.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.state.StateObject;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.function.Function;
+
+import static java.lang.Thread.holdsLock;
+import static java.util.Collections.singleton;
+import static 
org.apache.flink.runtime.state.track.TaskStateRegistryImpl.StateObjectIDExtractor.IDENTITY;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** @param <K> state object ID type (for {@link #distributedState}). */
+@ThreadSafe
+@Internal
+public class TaskStateRegistryImpl<K> implements TaskStateRegistry {
+    private static final Logger LOG = 
LoggerFactory.getLogger(TaskStateRegistryImpl.class);
+
+    private final TaskStateCleaner cleaner;
+    private final StateObjectIDExtractor<K> keyExtractor;
+
+    // Synchronization is to prevent issues inside backend registries, not 
just Map modification.
+    // E.g. backends may share State Entries and otherwise, would update them 
concurrently.
+    @GuardedBy("lock")
+    private final Map<String, BackendStateRegistry<K>> backendStateRegistries 
= new HashMap<>();
+
+    @GuardedBy("lock")
+    private final Set<K> usedState = new HashSet<>();
+
+    // Synchronization is just to prevent concurrent Map modification issues.
+    @GuardedBy("lock")
+    private final Set<K> distributedState = new HashSet<>();
+
+    private final Object lock = new Object();
+
+    TaskStateRegistryImpl(TaskStateCleaner cleaner, StateObjectIDExtractor<K> 
keyExtractor) {
+        this.cleaner = checkNotNull(cleaner);
+        this.keyExtractor = checkNotNull(keyExtractor);
+    }
+
+    @Override
+    public void stateUsed(Set<String> backendIds, Collection<StateObject> 
states) {
+        synchronized (lock) {
+            stateUsedInternal(backendIds, states);
+        }
+    }
+
+    private void stateUsedInternal(Set<String> backendIds, 
Collection<StateObject> states) {
+        checkState(holdsLock(lock));
+        List<StateEntry<K>> entries = toStateEntries(states, 
backendIds.size());
+        if (!entries.isEmpty()) {
+            for (StateEntry<K> e : entries) {
+                usedState.add(e.getKey());
+            }
+            for (String backendId : backendIds) {
+                withRegistry(backendId, registry -> 
registry.stateUsed(entries));
+            }
+        }
+    }
+
+    private List<StateEntry<K>> toStateEntries(Collection<StateObject> states, 
int numBackends) {
+        checkState(holdsLock(lock));
+        List<StateEntry<K>> entries = new ArrayList<>();
+        for (StateObject stateObject : states) {
+            for (Map.Entry<K, StateObject> entry : 
keyExtractor.apply(stateObject).entrySet()) {
+                K stateKey = entry.getKey();
+                StateObject state = entry.getValue();
+                if (shouldTrack(stateKey)) {
+                    entries.add(
+                            new StateEntry<>(
+                                    stateKey, state, numBackends, cleaner, 
usedState::remove));
+                }
+            }
+        }
+        return entries;
+    }
+
+    private boolean shouldTrack(K k) {
+        return !usedState.contains(k) && !distributedState.contains(k);
+    }
+
+    @Override
+    public void stateNotUsed(String backendId, StateObject state) {
+        Set<K> keys = keyExtractor.apply(state).keySet();
+        if (!keys.isEmpty()) {
+            withRegistry(backendId, registry -> registry.stateNotUsed(keys));
+        }
+    }
+
+    @Override
+    public void checkpointStarting(String backendId, long checkpointId, 
boolean isSavepoint) {
+        withRegistry(backendId, registry -> 
registry.checkpointStarting(checkpointId, isSavepoint));
+    }
+
+    @Override
+    public void snapshotTaken(
+            String backendId, StateObject state, long checkpointId, boolean 
trackStateUsage) {
+        withRegistry(
+                backendId,
+                registry -> {
+                    if (checkpointId > 
registry.getLastSnapshottedCheckpoint()) {
+                        if (trackStateUsage) {
+                            stateUsedInternal(singleton(backendId), 
singleton(state));
+                        }
+                        registry.snapshotTaken(checkpointId, state, 
trackStateUsage);
+                    }
+                });
+    }
+
+    @Override
+    public void checkpointSubsumed(String backendId, long checkpointId) {
+        withRegistry(backendId, registry -> 
registry.checkpointSubsumed(checkpointId));
+    }
+
+    @Override
+    public void checkpointAborted(String backendId, long checkpointId) {
+        withRegistry(backendId, registry -> 
registry.checkpointAborted(checkpointId));
+    }
+
+    @Override
+    public void close() throws Exception {
+        LOG.debug("Close");

Review comment:
       Do you mean it should be removed or some info added?




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