Skip to content

Commit

Permalink
[FLINK-4364] Introduce HeartbeatServices for the HeartbeatManager ins…
Browse files Browse the repository at this point in the history
…tantiation

The HeartbeatServices are used to create all services relevant for heartbeating. This
includes at the moment the creation of HeartbeatManager implementations which actively
send heartbeats and those which only respond to heartbeat requests.

Add comments
  • Loading branch information
tillrohrmann committed Mar 17, 2017
1 parent 0b3d5c2 commit 97ccc14
Show file tree
Hide file tree
Showing 18 changed files with 515 additions and 267 deletions.
Expand Up @@ -30,7 +30,7 @@ public class HeartbeatManagerOptions {

/** Time interval for requesting heartbeat from sender side */
public static final ConfigOption<Long> HEARTBEAT_INTERVAL =
key("heartbeat.sender.interval")
key("heartbeat.interval")
.defaultValue(10000L);

/** Timeout for requesting and receiving heartbeat for both sender and receiver sides */
Expand Down
@@ -0,0 +1,116 @@
/*
* 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.configuration.Configuration;
import org.apache.flink.configuration.HeartbeatManagerOptions;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.concurrent.ScheduledExecutor;
import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;

/**
* HeartbeatServices gives access to all services needed for heartbeating. This includes the
* creation of heartbeat receivers and heartbeat senders.
*/
public class HeartbeatServices {

/** Heartbeat interval for the created services */
protected final long heartbeatInterval;

/** Heartbeat timeout for the created services */
protected final long heartbeatTimeout;

public HeartbeatServices(long heartbeatInterval, long heartbeatTimeout) {
Preconditions.checkArgument(0L < heartbeatInterval, "The heartbeat interval must be larger than 0.");
Preconditions.checkArgument(heartbeatInterval <= heartbeatTimeout, "The heartbeat timeout should be larger or equal than the heartbeat timeout.");

this.heartbeatInterval = heartbeatInterval;
this.heartbeatTimeout = heartbeatTimeout;
}

/**
* Creates a heartbeat manager which does not actively send heartbeats.
*
* @param resourceId Resource Id which identifies the owner of the heartbeat manager
* @param heartbeatListener Listener which will be notified upon heartbeat timeouts for registered
* targets
* @param scheduledExecutor Scheduled executor to be used for scheduling heartbeat timeouts
* @param log Logger to be used for the logging
* @param <I> Type of the incoming payload
* @param <O> Type of the outgoing payload
* @return A new HeartbeatManager instance
*/
public <I, O> HeartbeatManager<I, O> createHeartbeatManager(
ResourceID resourceId,
HeartbeatListener<I, O> heartbeatListener,
ScheduledExecutor scheduledExecutor,
Logger log) {

return new HeartbeatManagerImpl<>(
heartbeatTimeout,
resourceId,
heartbeatListener,
scheduledExecutor,
scheduledExecutor,
log);
}

/**
* Creates a heartbeat manager which actively sends heartbeats to monitoring targets.
*
* @param resourceId Resource Id which identifies the owner of the heartbeat manager
* @param heartbeatListener Listener which will be notified upon heartbeat timeouts for registered
* targets
* @param scheduledExecutor Scheduled executor to be used for scheduling heartbeat timeouts
* @param log Logger to be used for the logging
* @param <I> Type of the incoming payload
* @param <O> Type of the outgoing payload
* @return A new HeartbeatManager instance which actively sends heartbeats
*/
public <I, O> HeartbeatManager<I, O> createHeartbeatManagerSender(
ResourceID resourceId,
HeartbeatListener<I, O> heartbeatListener,
ScheduledExecutor scheduledExecutor,
Logger log) {

return new HeartbeatManagerSenderImpl<>(
heartbeatInterval,
heartbeatTimeout,
resourceId,
heartbeatListener,
scheduledExecutor,
scheduledExecutor,
log);
}

/**
* Creates an HeartbeatServices instance from a {@link Configuration}.
*
* @param configuration Configuration to be used for the HeartbeatServices creation
* @return An HeartbeatServices instance created from the given configuration
*/
public static HeartbeatServices fromConfiguration(Configuration configuration) {
long heartbeatInterval = configuration.getLong(HeartbeatManagerOptions.HEARTBEAT_INTERVAL);

long heartbeatTimeout = configuration.getLong(HeartbeatManagerOptions.HEARTBEAT_TIMEOUT);

return new HeartbeatServices(heartbeatInterval, heartbeatTimeout);
}
}
Expand Up @@ -40,11 +40,12 @@ public TestingHeartbeatManagerImpl(
CountDownLatch waitLatch,
long heartbeatTimeoutIntervalMs,
ResourceID ownResourceID,
HeartbeatListener<I, O> heartbeatListener,
Executor executor,
ScheduledExecutor scheduledExecutor,
Logger log) {

super(heartbeatTimeoutIntervalMs, ownResourceID, executor, scheduledExecutor, log);
super(heartbeatTimeoutIntervalMs, ownResourceID, heartbeatListener, executor, scheduledExecutor, log);

this.waitLatch = waitLatch;
}
Expand Down
Expand Up @@ -38,11 +38,12 @@ public TestingHeartbeatManagerSenderImpl(
long heartbeatPeriod,
long heartbeatTimeout,
ResourceID ownResourceID,
HeartbeatListener<I, O> heartbeatListener,
Executor executor,
ScheduledExecutor scheduledExecutor,
Logger log) {

super(heartbeatPeriod, heartbeatTimeout, ownResourceID, executor, scheduledExecutor, log);
super(heartbeatPeriod, heartbeatTimeout, ownResourceID, heartbeatListener, executor, scheduledExecutor, log);

this.waitLatch = waitLatch;
}
Expand Down
Expand Up @@ -21,11 +21,10 @@
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.HeartbeatManagerOptions;
import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
import org.apache.flink.runtime.heartbeat.HeartbeatManagerSenderImpl;
import org.apache.flink.runtime.heartbeat.HeartbeatServices;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
Expand Down Expand Up @@ -88,31 +87,47 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F
// ------------------------------------------------------------------------

public JobManagerRunner(
final ResourceID resourceId,
final JobGraph jobGraph,
final Configuration configuration,
final RpcService rpcService,
final HighAvailabilityServices haServices,
final HeartbeatServices heartbeatServices,
final OnCompletionActions toNotifyOnComplete,
final FatalErrorHandler errorHandler) throws Exception
{
this(jobGraph, configuration, rpcService, haServices,
new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration)),
toNotifyOnComplete, errorHandler);
final FatalErrorHandler errorHandler) throws Exception {
this(
resourceId,
jobGraph,
configuration,
rpcService,
haServices,
heartbeatServices,
new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration)),
toNotifyOnComplete,
errorHandler);
}

public JobManagerRunner(
final ResourceID resourceId,
final JobGraph jobGraph,
final Configuration configuration,
final RpcService rpcService,
final HighAvailabilityServices haServices,
final HeartbeatServices heartbeatServices,
final MetricRegistry metricRegistry,
final OnCompletionActions toNotifyOnComplete,
final FatalErrorHandler errorHandler) throws Exception
{
this(jobGraph, configuration, rpcService, haServices,
JobManagerServices.fromConfiguration(configuration, haServices),
metricRegistry,
toNotifyOnComplete, errorHandler);
final FatalErrorHandler errorHandler) throws Exception {
this(
resourceId,
jobGraph,
configuration,
rpcService,
haServices,
heartbeatServices,
JobManagerServices.fromConfiguration(configuration, haServices),
metricRegistry,
toNotifyOnComplete,
errorHandler);
}

/**
Expand All @@ -127,15 +142,16 @@ public JobManagerRunner(
* required services could not be started, ot the Job could not be initialized.
*/
public JobManagerRunner(
final ResourceID resourceId,
final JobGraph jobGraph,
final Configuration configuration,
final RpcService rpcService,
final HighAvailabilityServices haServices,
final HeartbeatServices heartbeatServices,
final JobManagerServices jobManagerServices,
final MetricRegistry metricRegistry,
final OnCompletionActions toNotifyOnComplete,
final FatalErrorHandler errorHandler) throws Exception
{
final FatalErrorHandler errorHandler) throws Exception {

JobManagerMetricGroup jobManagerMetrics = null;

Expand Down Expand Up @@ -170,31 +186,22 @@ public JobManagerRunner(
this.runningJobsRegistry = haServices.getRunningJobsRegistry();
this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID());

// heartbeat manager last
final ResourceID resourceID = ResourceID.generate();
final HeartbeatManagerSenderImpl<Void, Void> jobManagerHeartbeatManager = new HeartbeatManagerSenderImpl<>(
configuration.getLong(HeartbeatManagerOptions.HEARTBEAT_INTERVAL),
configuration.getLong(HeartbeatManagerOptions.HEARTBEAT_TIMEOUT),
resourceID,
rpcService.getExecutor(),
rpcService.getScheduledExecutor(),
log);

// now start the JobManager
this.jobManager = new JobMaster(
jobGraph, configuration,
rpcService,
haServices,
jobManagerServices.executorService,
jobManagerServices.libraryCacheManager,
jobManagerServices.restartStrategyFactory,
jobManagerServices.rpcAskTimeout,
jobManagerMetrics,
resourceID,
jobManagerHeartbeatManager,
this,
this,
userCodeLoader);
resourceId,
jobGraph,
configuration,
rpcService,
haServices,
heartbeatServices,
jobManagerServices.executorService,
jobManagerServices.libraryCacheManager,
jobManagerServices.restartStrategyFactory,
jobManagerServices.rpcAskTimeout,
jobManagerMetrics,
this,
this,
userCodeLoader);
}
catch (Throwable t) {
// clean up everything
Expand Down

0 comments on commit 97ccc14

Please sign in to comment.