wanglijie95 commented on code in PR #20079: URL: https://github.com/apache/flink/pull/20079#discussion_r911836101
########## flink-runtime/src/main/java/org/apache/flink/runtime/blocklist/DefaultBlocklistHandler.java: ########## @@ -0,0 +1,181 @@ +/* + * 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.blocklist; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; + +import org.slf4j.Logger; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Default implementation of {@link BlocklistHandler}. */ +public class DefaultBlocklistHandler implements BlocklistHandler, AutoCloseable { + + private final Logger log; + + private final Map<ResourceID, String> taskManagerToNode; + + private final BlocklistTracker blocklistTracker; + + private final BlocklistContext blocklistContext; + + private final Set<BlocklistListener> blocklistListeners = new HashSet<>(); + + private final Time timeoutCheckInterval; + + private volatile ScheduledFuture<?> timeoutCheckFuture; + + private final ComponentMainThreadExecutor mainThreadExecutor; + + public DefaultBlocklistHandler( + BlocklistTracker blocklistTracker, + BlocklistContext blocklistContext, + Map<ResourceID, String> taskManagerToNode, + Time timeoutCheckInterval, + ComponentMainThreadExecutor mainThreadExecutor, + Logger log) { + this.blocklistTracker = checkNotNull(blocklistTracker); + this.blocklistContext = checkNotNull(blocklistContext); + this.taskManagerToNode = checkNotNull(taskManagerToNode); + this.timeoutCheckInterval = checkNotNull(timeoutCheckInterval); + this.mainThreadExecutor = checkNotNull(mainThreadExecutor); + this.log = checkNotNull(log); + + scheduleTimeoutCheck(); + } + + private void scheduleTimeoutCheck() { + this.timeoutCheckFuture = + mainThreadExecutor.schedule( + () -> { + removeTimeoutNodes(); + scheduleTimeoutCheck(); + }, + timeoutCheckInterval.toMilliseconds(), + TimeUnit.MILLISECONDS); + } + + private void removeTimeoutNodes() { + Collection<BlockedNode> removedNodes = + blocklistTracker.removeTimeoutNodes(System.currentTimeMillis()); + if (!removedNodes.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug( + "Timeout removed blocked nodes: {}, details {}. " + + "Total blocked nodes currently: {}, details: {}.", + removedNodes.size(), + removedNodes, + blocklistTracker.getAllBlockedNodes().size(), + blocklistTracker.getAllBlockedNodes()); + } else { + log.info( + "Timeout removed blocked nodes: {}." + + " Total blocked nodes currently: {}.", + removedNodes.size(), + blocklistTracker.getAllBlockedNodes().size()); + } + blocklistContext.unBlockResources(removedNodes); + } + } + + private void assertRunningInMainThread() { + mainThreadExecutor.assertRunningInMainThread(); + } + + @Override + public void addNewBlockedNodes(Collection<BlockedNode> newNodes) { + assertRunningInMainThread(); + + Collection<BlockedNode> newlyAddedOrMerged = blocklistTracker.addNewBlockedNodes(newNodes); + if (!newlyAddedOrMerged.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug( + "Newly added/merged blocked nodes: {}, details: {}." + + " Total blocked nodes currently: {}, details: {}.", + newlyAddedOrMerged.size(), + newlyAddedOrMerged, + blocklistTracker.getAllBlockedNodes().size(), + blocklistTracker.getAllBlockedNodes()); + } else { + log.info( + "Newly added/merged blocked nodes: {}." + + " Total blocked nodes currently: {}.", + newlyAddedOrMerged.size(), + blocklistTracker.getAllBlockedNodes().size()); + } + + blocklistListeners.forEach( + listener -> listener.notifyNewBlockedNodes(newlyAddedOrMerged)); + blocklistContext.blockResources(newlyAddedOrMerged); + } + } + + @Override + public boolean isBlockedTaskManager(ResourceID taskManagerId) { + assertRunningInMainThread(); + + return taskManagerToNode.containsKey(taskManagerId) Review Comment: You are right, we never expect it fails to retrieve the node id of a task manager. -- 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