Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r81348543
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerImpl.java
 ---
    @@ -0,0 +1,328 @@
    +/*
    + * 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.heartbeat;
    +
    +import org.apache.flink.runtime.clusterframework.types.ResourceID;
    +import org.apache.flink.runtime.concurrent.AcceptFunction;
    +import org.apache.flink.runtime.concurrent.Future;
    +import org.apache.flink.util.Preconditions;
    +import org.slf4j.Logger;
    +
    +import javax.annotation.concurrent.ThreadSafe;
    +import java.util.Collection;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * Heartbeat manager implementation. The heartbeat manager maintains a map 
of heartbeat monitors
    + * and resource IDs. Each monitor will be updated when a new heartbeat of 
the associated machine has
    + * been received. If the monitor detects that a heartbeat has timed out, 
it will notify the
    + * {@link HeartbeatListener} about it. A heartbeat times out iff no 
heartbeat signal has been
    + * received within a given timeout interval.
    + *
    + * @param <I> Type of the incoming heartbeat payload
    + * @param <O> Type of the outgoing heartbeat payload
    + */
    +@ThreadSafe
    +public class HeartbeatManagerImpl<I, O> implements HeartbeatManager<I, O>, 
HeartbeatTarget<I> {
    +
    +   /** Heartbeat timeout interval in milli seconds */
    +   private final long heartbeatTimeoutIntervalMs;
    +
    +   /** Resource ID which is used to mark one own's heartbeat signals */
    +   private final ResourceID ownResourceID;
    +
    +   /** Executor service used to run heartbeat timeout notifications */
    +   private final ScheduledExecutorService scheduledExecutorService;
    +
    +   protected final Logger log;
    +
    +   /** Map containing the heartbeat monitors associated with the 
respective resource ID */
    +   private final ConcurrentHashMap<ResourceID, 
HeartbeatManagerImpl.HeartbeatMonitor<O>> heartbeatTargets;
    +
    +   /** Execution context used to run future callbacks */
    +   private final Executor executor;
    +
    +   /** Heartbeat listener with which the heartbeat manager has been 
associated */
    +   private HeartbeatListener<I, O> heartbeatListener;
    +
    +   /** Running state of the heartbeat manager */
    +   protected boolean stopped;
    +
    +   public HeartbeatManagerImpl(
    +           long heartbeatTimeoutIntervalMs,
    +           ResourceID ownResourceID,
    +           Executor executor,
    +           ScheduledExecutorService scheduledExecutorService,
    +           Logger log) {
    +           Preconditions.checkArgument(heartbeatTimeoutIntervalMs > 0L, 
"The heartbeat timeout has to be larger than 0.");
    +
    +           this.heartbeatTimeoutIntervalMs = heartbeatTimeoutIntervalMs;
    +           this.ownResourceID = Preconditions.checkNotNull(ownResourceID);
    +           this.scheduledExecutorService = 
Preconditions.checkNotNull(scheduledExecutorService);
    +           this.log = Preconditions.checkNotNull(log);
    +           this.executor = Preconditions.checkNotNull(executor);
    +           this.heartbeatTargets = new ConcurrentHashMap<>(16);
    +
    +           stopped = true;
    +   }
    +
    +   
//----------------------------------------------------------------------------------------------
    +   // Getters
    +   
//----------------------------------------------------------------------------------------------
    +
    +   ResourceID getOwnResourceID() {
    +           return ownResourceID;
    +   }
    +
    +   Executor getExecutor() {
    +           return executor;
    +   }
    +
    +   HeartbeatListener<I, O> getHeartbeatListener() {
    +           return heartbeatListener;
    +   }
    +
    +   Collection<HeartbeatManagerImpl.HeartbeatMonitor<O>> 
getHeartbeatTargets() {
    +           return heartbeatTargets.values();
    +   }
    +
    +   
//----------------------------------------------------------------------------------------------
    +   // HeartbeatManager methods
    +   
//----------------------------------------------------------------------------------------------
    +
    +   @Override
    +   public void monitorTarget(ResourceID resourceID, HeartbeatTarget<O> 
heartbeatTarget) {
    +           if (!stopped) {
    +                   if (heartbeatTargets.containsKey(resourceID)) {
    +                           log.info("The target with resource ID {} is 
already been monitored.", resourceID);
    +                   } else {
    +                           HeartbeatManagerImpl.HeartbeatMonitor<O> 
heartbeatMonitor = new HeartbeatManagerImpl.HeartbeatMonitor<>(
    +                                   resourceID,
    +                                   heartbeatTarget,
    +                                   scheduledExecutorService,
    +                                   heartbeatListener,
    +                                   heartbeatTimeoutIntervalMs);
    +
    +                           heartbeatTargets.put(
    +                                   resourceID,
    +                                   heartbeatMonitor);
    +
    +                           // check if we have stopped in the meantime 
(concurrent stop operation)
    +                           if (stopped) {
    +                                   heartbeatMonitor.cancel();
    +
    +                                   heartbeatTargets.remove(resourceID);
    +                           }
    --- End diff --
    
    We have to check after we've put the element in the concurrent hash map, 
because the stop operation can happen concurrently. If the stop took place 
after we've inserted it, then it will be removed by the stop call. If the stop 
took place right before we've inserted the element, then we have to clean it up 
ourselves.
    I tried to avoid locking in order to make access non-blocking.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to