rkhachatryan commented on code in PR #19550:
URL: https://github.com/apache/flink/pull/19550#discussion_r858732789


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/ChangelogRegistryImpl.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.state.PhysicalStateHandleID;
+import org.apache.flink.runtime.state.StreamStateHandle;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.Executor;
+
+@Internal
+@ThreadSafe
+class ChangelogRegistryImpl implements ChangelogRegistry {
+    private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogRegistryImpl.class);
+
+    private final Map<PhysicalStateHandleID, Set<UUID>> entries = new 
ConcurrentHashMap<>();
+    private final Executor executor;
+
+    public ChangelogRegistryImpl(Executor executor) {
+        this.executor = executor;
+    }
+
+    @Override
+    public void startTracking(StreamStateHandle handle, Set<UUID> backendIDs) {
+        LOG.debug(
+                "start tracking state, key: {}, state: {}",
+                handle.getStreamStateHandleID(),
+                handle);
+        entries.put(handle.getStreamStateHandleID(), new 
CopyOnWriteArraySet<>(backendIDs));
+    }
+
+    @Override
+    public void stopTracking(StreamStateHandle handle) {
+        LOG.debug(
+                "stop tracking state, key: {}, state: {}", 
handle.getStreamStateHandleID(), handle);
+        entries.remove(handle.getStreamStateHandleID());
+    }
+
+    @Override
+    public void notUsed(StreamStateHandle handle, UUID backendId) {
+        PhysicalStateHandleID key = handle.getStreamStateHandleID();
+        LOG.debug("backend {} not using state, key: {}, state: {}", backendId, 
key, handle);
+        Set<UUID> backends = entries.get(key);
+        if (backends == null) {
+            LOG.warn("backend {} was not using state, key: {}, state: {}", 
backendId, key, handle);

Review Comment:
   In the 2nd case, the state is **already** not tracked for the given backend.
   As this is a normal case, I'll remove the log message (this might happen 
after some other backend calls `stopTracking`).



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to