You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by tillrohrmann <gi...@git.apache.org> on 2016/08/29 13:29:32 UTC

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

GitHub user tillrohrmann opened a pull request:

    https://github.com/apache/flink/pull/2435

    [FLINK-4478] [flip-6] Add HeartbeatManager

    Adds a `HeartbeatManager` abstraction which can monitor heartbeat targets. Whenever no heartbeat signal has been received for a given heartbeat timeout interval, the heartbeat manager will issue a heartbeat timeout notification.
    
    The `HeartbeatManager` abstraction works completely independent of the RPC service. Furthermore, it reuses the same heartbeat logic for the sending and receiving side of the heartbeat.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tillrohrmann/flink heartbeating

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2435.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2435
    
----
commit 8db510fdd20b9eef94fbb07be2ac3f05dd2509a3
Author: Till Rohrmann <tr...@apache.org>
Date:   2016-08-25T12:05:07Z

    [FLINK-4478] [flip-6] Add HeartbeatManager
    
    Add a heartbeat manager abstraction which can monitor heartbeat targets. Whenever
    no heartbeat signal has been received for a heartbeat timeout interval, the
    heartbeat manager will issue a heartbeat timeout notification.

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/2435
  
    I've rebased the PR onto the latest flip-6 branch.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the issue:

    https://github.com/apache/flink/pull/2435
  
    Could you rebase this again? I think we should merge this soon.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/2435
  
    I've rebased the PR on the latest Flip-6 branch.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by beyond1920 <gi...@git.apache.org>.
Github user beyond1920 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r76929812
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java ---
    @@ -0,0 +1,61 @@
    +/*
    + * 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 scala.concurrent.Future;
    +
    +/**
    + * Interface for the interaction with the {@link HeartbeatManager}. The heartbeat listener is used
    + * for the following things:
    + * <p>
    + * <ul>
    + *     <il>Notifications about heartbeat timeouts</il>
    + *     <li>Payload reports of incoming heartbeats</li>
    + *     <li>Retrieval of payloads for outgoing heartbeats</li>
    + * </ul>
    + * @param <I> Type of the incoming payload
    + * @param <O> Type of the outgoing payload
    + */
    +public interface HeartbeatListener<I, O> {
    +
    +	/**
    +	 * Callback which is called if a heartbeat for the machine identified by the given resource
    +	 * ID times out.
    +	 *
    +	 * @param resourceID Resource ID of the machine whose heartbeat has timed out
    +	 */
    +	void notifyHeartbeatTimeout(ResourceID resourceID);
    +
    +	/**
    +	 * Callback which is called whenever a heartbeat with an associated payload is received. The
    +	 * carried payload is given to this method.
    +	 *
    +	 * @param payload Payload of the received heartbeat
    +	 */
    +	void reportPayload(I payload);
    --- End diff --
    
    maybe we should add ResourceID param to identify which sender send this incoming message


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by beyond1920 <gi...@git.apache.org>.
Github user beyond1920 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r76938462
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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 scala.concurrent.Future;
    +
    +/**
    + * Interface for the interaction with the {@link HeartbeatManager}. The heartbeat listener is used
    + * for the following things:
    + * <p>
    + * <ul>
    + *     <il>Notifications about heartbeat timeouts</il>
    + *     <li>Payload reports of incoming heartbeats</li>
    + *     <li>Retrieval of payloads for outgoing heartbeats</li>
    + * </ul>
    + * @param <I> Type of the incoming payload
    + * @param <O> Type of the outgoing payload
    + */
    +public interface HeartbeatListener<I, O> {
    +
    +	/**
    +	 * Callback which is called if a heartbeat for the machine identified by the given resource
    +	 * ID times out.
    +	 *
    +	 * @param resourceID Resource ID of the machine whose heartbeat has timed out
    +	 */
    +	void notifyHeartbeatTimeout(ResourceID resourceID);
    --- End diff --
    
    It seems that JobManager and ResourceManager don't have ResourceID, only TaskExecutor has ResourceID. Would it be more proper to use something else to identify the heartbeat target?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r81313363
  
    --- 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);
    +				}
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void unmonitorTarget(ResourceID resourceID) {
    +		if (!stopped) {
    +			HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor = heartbeatTargets.remove(resourceID);
    +
    +			if (heartbeatMonitor != null) {
    +				heartbeatMonitor.cancel();
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void start(HeartbeatListener<I, O> heartbeatListener) {
    +		Preconditions.checkState(stopped, "Cannot start an already started heartbeat manager.");
    +
    +		stopped = false;
    +
    +		this.heartbeatListener = Preconditions.checkNotNull(heartbeatListener);
    +	}
    +
    +	@Override
    +	public void stop() {
    +		stopped = true;
    +
    +		for (HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor : heartbeatTargets.values()) {
    +			heartbeatMonitor.cancel();
    +		}
    +
    +		heartbeatTargets.clear();
    +	}
    +
    +	//----------------------------------------------------------------------------------------------
    +	// HeartbeatTarget methods
    +	//----------------------------------------------------------------------------------------------
    +
    +	@Override
    +	public void sendHeartbeat(ResourceID resourceID, I payload) {
    +		if (!stopped) {
    +			log.debug("Received heartbeat from {}.", resourceID);
    +			reportHeartbeat(resourceID);
    +
    +			if (payload != null) {
    +				heartbeatListener.reportPayload(resourceID, payload);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void requestHeartbeat(ResourceID resourceID, I payload) {
    +		if (!stopped) {
    +			log.debug("Received heartbeat request from {}.", resourceID);
    +
    +			final HeartbeatTarget<O> heartbeatTarget = reportHeartbeat(resourceID);
    +
    +			if (heartbeatTarget != null) {
    +				if (payload != null) {
    +					heartbeatListener.reportPayload(resourceID, payload);
    +				}
    +
    +				Future<O> futurePayload = heartbeatListener.retrievePayload();
    +
    +				if (futurePayload != null) {
    +					futurePayload.thenAcceptAsync(new AcceptFunction<O>() {
    +						@Override
    +						public void accept(O retrievedPayload) {
    +							heartbeatTarget.sendHeartbeat(getOwnResourceID(), retrievedPayload);
    +						}
    +					}, executor);
    +				} else {
    +					heartbeatTarget.sendHeartbeat(ownResourceID, null);
    +				}
    +			}
    +		}
    +	}
    +
    +	HeartbeatTarget<O> reportHeartbeat(ResourceID resourceID) {
    +		if (heartbeatTargets.containsKey(resourceID)) {
    +			HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor = heartbeatTargets.get(resourceID);
    +			heartbeatMonitor.reportHeartbeat();
    +
    +			return heartbeatMonitor.getHeartbeatTarget();
    +		} else {
    +			return null;
    +		}
    +	}
    +
    +	//----------------------------------------------------------------------------------------------
    +	// Utility classes
    +	//----------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Heartbeat monitor which manages the heartbeat state of the associated heartbeat target. The
    +	 * monitor notifies the {@link HeartbeatListener} whenever it has not seen a heartbeat signal
    +	 * in the specified heartbeat timeout interval. Each heartbeat signal resets this timer.
    +	 *
    +	 * @param <O> Type of the payload being sent to the associated heartbeat target
    +	 */
    +	static class HeartbeatMonitor<O> implements Runnable {
    +
    +		/** Resource ID of the monitored heartbeat target */
    +		private final ResourceID resourceID;
    +
    +		/** Associated heartbeat target */
    +		private final HeartbeatTarget<O> heartbeatTarget;
    +
    +		private final ScheduledExecutorService scheduledExecutorService;
    +
    +		/** Listener which is notified about heartbeat timeouts */
    +		private final HeartbeatListener<?, ?> heartbeatListener;
    +
    +		/** Maximum heartbeat timeout interval */
    +		private final long heartbeatTimeoutIntervalMs;
    +
    +		private volatile ScheduledFuture<?> futureTimeout;
    +
    +		private final AtomicReference<State> state = new AtomicReference<>(State.RUNNING);
    +
    +		HeartbeatMonitor(
    +			ResourceID resourceID,
    +			HeartbeatTarget<O> heartbeatTarget,
    +			ScheduledExecutorService scheduledExecutorService,
    +			HeartbeatListener<?, O> heartbeatListener,
    +			long heartbeatTimeoutIntervalMs) {
    +
    +			this.resourceID = Preconditions.checkNotNull(resourceID);
    +			this.heartbeatTarget = Preconditions.checkNotNull(heartbeatTarget);
    +			this.scheduledExecutorService = Preconditions.checkNotNull(scheduledExecutorService);
    +			this.heartbeatListener = Preconditions.checkNotNull(heartbeatListener);
    +
    +			Preconditions.checkArgument(heartbeatTimeoutIntervalMs >= 0L, "The heartbeat timeout interval has to be larger than 0.");
    +			this.heartbeatTimeoutIntervalMs = heartbeatTimeoutIntervalMs;
    +
    +			resetHeartbeatTimeout(heartbeatTimeoutIntervalMs);
    +		}
    +
    +		HeartbeatTarget<O> getHeartbeatTarget() {
    +			return heartbeatTarget;
    +		}
    +
    +		void reportHeartbeat() {
    +			resetHeartbeatTimeout(heartbeatTimeoutIntervalMs);
    +		}
    +
    +		void resetHeartbeatTimeout(long heartbeatTimeout) {
    +			if (state.get() == State.RUNNING) {
    +				cancelTimeout();
    +
    +				futureTimeout = scheduledExecutorService.schedule(this, heartbeatTimeout, TimeUnit.MILLISECONDS);
    +
    +				// Double check for concurrent accesses (e.g. a firing of the scheduled future)
    +				if (state.get() != State.RUNNING) {
    +					cancelTimeout();
    +				}
    --- End diff --
    
    This looks like an optimization. Is is really necessary? We can still change state after we have checked.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r76937451
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java ---
    @@ -0,0 +1,61 @@
    +/*
    + * 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 scala.concurrent.Future;
    +
    +/**
    + * Interface for the interaction with the {@link HeartbeatManager}. The heartbeat listener is used
    + * for the following things:
    + * <p>
    + * <ul>
    + *     <il>Notifications about heartbeat timeouts</il>
    + *     <li>Payload reports of incoming heartbeats</li>
    + *     <li>Retrieval of payloads for outgoing heartbeats</li>
    + * </ul>
    + * @param <I> Type of the incoming payload
    + * @param <O> Type of the outgoing payload
    + */
    +public interface HeartbeatListener<I, O> {
    +
    +	/**
    +	 * Callback which is called if a heartbeat for the machine identified by the given resource
    +	 * ID times out.
    +	 *
    +	 * @param resourceID Resource ID of the machine whose heartbeat has timed out
    +	 */
    +	void notifyHeartbeatTimeout(ResourceID resourceID);
    +
    +	/**
    +	 * Callback which is called whenever a heartbeat with an associated payload is received. The
    +	 * carried payload is given to this method.
    +	 *
    +	 * @param payload Payload of the received heartbeat
    +	 */
    +	void reportPayload(I payload);
    --- End diff --
    
    Yes, good point. Will add it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r81309944
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java ---
    @@ -43,12 +42,9 @@
     import org.junit.BeforeClass;
     import org.junit.Test;
     import org.mockito.Mockito;
    -import org.mockito.invocation.InvocationOnMock;
    -import org.mockito.stubbing.Answer;
     
     import java.util.Collections;
     import java.util.UUID;
    -import java.util.concurrent.CompletableFuture;
    --- End diff --
    
    These changes are unrelated. I'm cleaning up the imports in #2571. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r76939200
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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 scala.concurrent.Future;
    +
    +/**
    + * Interface for the interaction with the {@link HeartbeatManager}. The heartbeat listener is used
    + * for the following things:
    + * <p>
    + * <ul>
    + *     <il>Notifications about heartbeat timeouts</il>
    + *     <li>Payload reports of incoming heartbeats</li>
    + *     <li>Retrieval of payloads for outgoing heartbeats</li>
    + * </ul>
    + * @param <I> Type of the incoming payload
    + * @param <O> Type of the outgoing payload
    + */
    +public interface HeartbeatListener<I, O> {
    +
    +	/**
    +	 * Callback which is called if a heartbeat for the machine identified by the given resource
    +	 * ID times out.
    +	 *
    +	 * @param resourceID Resource ID of the machine whose heartbeat has timed out
    +	 */
    +	void notifyHeartbeatTimeout(ResourceID resourceID);
    --- End diff --
    
    We need something unique. If they don't have one, then we should think about either assigning them one or to use something different. Should be easy to change in the `HeartbeatManager` implementation.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann closed the pull request at:

    https://github.com/apache/flink/pull/2435


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r81314113
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java ---
    @@ -43,12 +42,9 @@
     import org.junit.BeforeClass;
     import org.junit.Test;
     import org.mockito.Mockito;
    -import org.mockito.invocation.InvocationOnMock;
    -import org.mockito.stubbing.Answer;
     
     import java.util.Collections;
     import java.util.UUID;
    -import java.util.concurrent.CompletableFuture;
    --- End diff --
    
    You can leave them though. Not really an issue.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by tillrohrmann <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r81312934
  
    --- 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);
    +				}
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void unmonitorTarget(ResourceID resourceID) {
    +		if (!stopped) {
    +			HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor = heartbeatTargets.remove(resourceID);
    +
    +			if (heartbeatMonitor != null) {
    +				heartbeatMonitor.cancel();
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void start(HeartbeatListener<I, O> heartbeatListener) {
    +		Preconditions.checkState(stopped, "Cannot start an already started heartbeat manager.");
    +
    +		stopped = false;
    +
    +		this.heartbeatListener = Preconditions.checkNotNull(heartbeatListener);
    +	}
    +
    +	@Override
    +	public void stop() {
    +		stopped = true;
    +
    +		for (HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor : heartbeatTargets.values()) {
    +			heartbeatMonitor.cancel();
    +		}
    +
    +		heartbeatTargets.clear();
    +	}
    +
    +	//----------------------------------------------------------------------------------------------
    +	// HeartbeatTarget methods
    +	//----------------------------------------------------------------------------------------------
    +
    +	@Override
    +	public void sendHeartbeat(ResourceID resourceID, I payload) {
    +		if (!stopped) {
    +			log.debug("Received heartbeat from {}.", resourceID);
    +			reportHeartbeat(resourceID);
    +
    +			if (payload != null) {
    +				heartbeatListener.reportPayload(resourceID, payload);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void requestHeartbeat(ResourceID resourceID, I payload) {
    +		if (!stopped) {
    +			log.debug("Received heartbeat request from {}.", resourceID);
    +
    +			final HeartbeatTarget<O> heartbeatTarget = reportHeartbeat(resourceID);
    +
    +			if (heartbeatTarget != null) {
    +				if (payload != null) {
    +					heartbeatListener.reportPayload(resourceID, payload);
    +				}
    +
    +				Future<O> futurePayload = heartbeatListener.retrievePayload();
    +
    +				if (futurePayload != null) {
    +					futurePayload.thenAcceptAsync(new AcceptFunction<O>() {
    +						@Override
    +						public void accept(O retrievedPayload) {
    +							heartbeatTarget.sendHeartbeat(getOwnResourceID(), retrievedPayload);
    +						}
    +					}, executor);
    +				} else {
    +					heartbeatTarget.sendHeartbeat(ownResourceID, null);
    +				}
    +			}
    +		}
    +	}
    +
    +	HeartbeatTarget<O> reportHeartbeat(ResourceID resourceID) {
    +		if (heartbeatTargets.containsKey(resourceID)) {
    +			HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor = heartbeatTargets.get(resourceID);
    +			heartbeatMonitor.reportHeartbeat();
    +
    +			return heartbeatMonitor.getHeartbeatTarget();
    +		} else {
    +			return null;
    +		}
    +	}
    +
    +	//----------------------------------------------------------------------------------------------
    +	// Utility classes
    +	//----------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Heartbeat monitor which manages the heartbeat state of the associated heartbeat target. The
    +	 * monitor notifies the {@link HeartbeatListener} whenever it has not seen a heartbeat signal
    +	 * in the specified heartbeat timeout interval. Each heartbeat signal resets this timer.
    +	 *
    +	 * @param <O> Type of the payload being sent to the associated heartbeat target
    +	 */
    +	static class HeartbeatMonitor<O> implements Runnable {
    +
    +		/** Resource ID of the monitored heartbeat target */
    +		private final ResourceID resourceID;
    +
    +		/** Associated heartbeat target */
    +		private final HeartbeatTarget<O> heartbeatTarget;
    +
    +		private final ScheduledExecutorService scheduledExecutorService;
    +
    +		/** Listener which is notified about heartbeat timeouts */
    +		private final HeartbeatListener<?, ?> heartbeatListener;
    +
    +		/** Maximum heartbeat timeout interval */
    +		private final long heartbeatTimeoutIntervalMs;
    +
    +		private volatile ScheduledFuture<?> futureTimeout;
    +
    +		private final AtomicReference<State> state = new AtomicReference<>(State.RUNNING);
    +
    +		HeartbeatMonitor(
    +			ResourceID resourceID,
    +			HeartbeatTarget<O> heartbeatTarget,
    +			ScheduledExecutorService scheduledExecutorService,
    +			HeartbeatListener<?, O> heartbeatListener,
    +			long heartbeatTimeoutIntervalMs) {
    +
    +			this.resourceID = Preconditions.checkNotNull(resourceID);
    +			this.heartbeatTarget = Preconditions.checkNotNull(heartbeatTarget);
    +			this.scheduledExecutorService = Preconditions.checkNotNull(scheduledExecutorService);
    +			this.heartbeatListener = Preconditions.checkNotNull(heartbeatListener);
    +
    +			Preconditions.checkArgument(heartbeatTimeoutIntervalMs >= 0L, "The heartbeat timeout interval has to be larger than 0.");
    +			this.heartbeatTimeoutIntervalMs = heartbeatTimeoutIntervalMs;
    +
    +			resetHeartbeatTimeout(heartbeatTimeoutIntervalMs);
    +		}
    +
    +		HeartbeatTarget<O> getHeartbeatTarget() {
    +			return heartbeatTarget;
    +		}
    +
    +		void reportHeartbeat() {
    +			resetHeartbeatTimeout(heartbeatTimeoutIntervalMs);
    +		}
    +
    +		void resetHeartbeatTimeout(long heartbeatTimeout) {
    +			if (state.get() == State.RUNNING) {
    +				cancelTimeout();
    +
    +				futureTimeout = scheduledExecutorService.schedule(this, heartbeatTimeout, TimeUnit.MILLISECONDS);
    +
    +				// Double check for concurrent accesses (e.g. a firing of the scheduled future)
    +				if (state.get() != State.RUNNING) {
    +					cancelTimeout();
    +				}
    +			}
    +		}
    +
    +		void cancel() {
    +			// we can only cancel if we are in state running
    +			if (state.compareAndSet(State.RUNNING, State.CANCELED)) {
    +				cancelTimeout();
    +			}
    +		}
    +
    +		private void cancelTimeout() {
    +			if (futureTimeout != null) {
    +				futureTimeout.cancel(true);
    +			}
    +		}
    +
    +		public boolean isCanceled() {
    +			return state.get() == State.CANCELED;
    +		}
    +
    +		@Override
    +		public void run() {
    +			// The heartbeat has timed out if we're in state running
    +			if (state.compareAndSet(State.RUNNING, State.TIMEOUT)) {
    +				heartbeatListener.notifyHeartbeatTimeout(resourceID);
    --- End diff --
    
    Took me a while to figure out this is called by the executor after a timeout. Makes sense.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/2435
  
    Thanks for the review @mxm. Concerning your questions:
    
    1. Double checking is necessary due to concurrent accesses. Of course one could also use locks but that way it is less blocking.
    2. I'll add a test for the `unmonitoring`.
    
    After adding the additional test, I'll merge this PR.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r81310859
  
    --- 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 --
    
    What if the `stopped = true` is set here? It seems kind of arbitrary to check at this point in time. Isn't it enough to put the `heartbeatMonitor` into the ConcurrentHashMap and remove the above lines? If we really want to be sure that no concurrency occurs while adding a new `HeartbeatMonitor`, why don't we synchronize on a lock then?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2435: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2435#discussion_r81348862
  
    --- 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);
    +				}
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void unmonitorTarget(ResourceID resourceID) {
    +		if (!stopped) {
    +			HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor = heartbeatTargets.remove(resourceID);
    +
    +			if (heartbeatMonitor != null) {
    +				heartbeatMonitor.cancel();
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void start(HeartbeatListener<I, O> heartbeatListener) {
    +		Preconditions.checkState(stopped, "Cannot start an already started heartbeat manager.");
    +
    +		stopped = false;
    +
    +		this.heartbeatListener = Preconditions.checkNotNull(heartbeatListener);
    +	}
    +
    +	@Override
    +	public void stop() {
    +		stopped = true;
    +
    +		for (HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor : heartbeatTargets.values()) {
    +			heartbeatMonitor.cancel();
    +		}
    +
    +		heartbeatTargets.clear();
    +	}
    +
    +	//----------------------------------------------------------------------------------------------
    +	// HeartbeatTarget methods
    +	//----------------------------------------------------------------------------------------------
    +
    +	@Override
    +	public void sendHeartbeat(ResourceID resourceID, I payload) {
    +		if (!stopped) {
    +			log.debug("Received heartbeat from {}.", resourceID);
    +			reportHeartbeat(resourceID);
    +
    +			if (payload != null) {
    +				heartbeatListener.reportPayload(resourceID, payload);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void requestHeartbeat(ResourceID resourceID, I payload) {
    +		if (!stopped) {
    +			log.debug("Received heartbeat request from {}.", resourceID);
    +
    +			final HeartbeatTarget<O> heartbeatTarget = reportHeartbeat(resourceID);
    +
    +			if (heartbeatTarget != null) {
    +				if (payload != null) {
    +					heartbeatListener.reportPayload(resourceID, payload);
    +				}
    +
    +				Future<O> futurePayload = heartbeatListener.retrievePayload();
    +
    +				if (futurePayload != null) {
    +					futurePayload.thenAcceptAsync(new AcceptFunction<O>() {
    +						@Override
    +						public void accept(O retrievedPayload) {
    +							heartbeatTarget.sendHeartbeat(getOwnResourceID(), retrievedPayload);
    +						}
    +					}, executor);
    +				} else {
    +					heartbeatTarget.sendHeartbeat(ownResourceID, null);
    +				}
    +			}
    +		}
    +	}
    +
    +	HeartbeatTarget<O> reportHeartbeat(ResourceID resourceID) {
    +		if (heartbeatTargets.containsKey(resourceID)) {
    +			HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor = heartbeatTargets.get(resourceID);
    +			heartbeatMonitor.reportHeartbeat();
    +
    +			return heartbeatMonitor.getHeartbeatTarget();
    +		} else {
    +			return null;
    +		}
    +	}
    +
    +	//----------------------------------------------------------------------------------------------
    +	// Utility classes
    +	//----------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Heartbeat monitor which manages the heartbeat state of the associated heartbeat target. The
    +	 * monitor notifies the {@link HeartbeatListener} whenever it has not seen a heartbeat signal
    +	 * in the specified heartbeat timeout interval. Each heartbeat signal resets this timer.
    +	 *
    +	 * @param <O> Type of the payload being sent to the associated heartbeat target
    +	 */
    +	static class HeartbeatMonitor<O> implements Runnable {
    +
    +		/** Resource ID of the monitored heartbeat target */
    +		private final ResourceID resourceID;
    +
    +		/** Associated heartbeat target */
    +		private final HeartbeatTarget<O> heartbeatTarget;
    +
    +		private final ScheduledExecutorService scheduledExecutorService;
    +
    +		/** Listener which is notified about heartbeat timeouts */
    +		private final HeartbeatListener<?, ?> heartbeatListener;
    +
    +		/** Maximum heartbeat timeout interval */
    +		private final long heartbeatTimeoutIntervalMs;
    +
    +		private volatile ScheduledFuture<?> futureTimeout;
    +
    +		private final AtomicReference<State> state = new AtomicReference<>(State.RUNNING);
    +
    +		HeartbeatMonitor(
    +			ResourceID resourceID,
    +			HeartbeatTarget<O> heartbeatTarget,
    +			ScheduledExecutorService scheduledExecutorService,
    +			HeartbeatListener<?, O> heartbeatListener,
    +			long heartbeatTimeoutIntervalMs) {
    +
    +			this.resourceID = Preconditions.checkNotNull(resourceID);
    +			this.heartbeatTarget = Preconditions.checkNotNull(heartbeatTarget);
    +			this.scheduledExecutorService = Preconditions.checkNotNull(scheduledExecutorService);
    +			this.heartbeatListener = Preconditions.checkNotNull(heartbeatListener);
    +
    +			Preconditions.checkArgument(heartbeatTimeoutIntervalMs >= 0L, "The heartbeat timeout interval has to be larger than 0.");
    +			this.heartbeatTimeoutIntervalMs = heartbeatTimeoutIntervalMs;
    +
    +			resetHeartbeatTimeout(heartbeatTimeoutIntervalMs);
    +		}
    +
    +		HeartbeatTarget<O> getHeartbeatTarget() {
    +			return heartbeatTarget;
    +		}
    +
    +		void reportHeartbeat() {
    +			resetHeartbeatTimeout(heartbeatTimeoutIntervalMs);
    +		}
    +
    +		void resetHeartbeatTimeout(long heartbeatTimeout) {
    +			if (state.get() == State.RUNNING) {
    +				cancelTimeout();
    +
    +				futureTimeout = scheduledExecutorService.schedule(this, heartbeatTimeout, TimeUnit.MILLISECONDS);
    +
    +				// Double check for concurrent accesses (e.g. a firing of the scheduled future)
    +				if (state.get() != State.RUNNING) {
    +					cancelTimeout();
    +				}
    --- End diff --
    
    If the state changes after we've created the new timeout, this means that we've already fired or canceled in the meantime. That's why we have to cancel the new scheduled timeout action. Of course, this is more of a corner case.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---