flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-4478) Implement heartbeat logic
Date Wed, 31 Aug 2016 06:18:20 GMT

    [ https://issues.apache.org/jira/browse/FLINK-4478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15451297#comment-15451297
] 

ASF GitHub Bot commented on FLINK-4478:
---------------------------------------

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


> Implement heartbeat logic
> -------------------------
>
>                 Key: FLINK-4478
>                 URL: https://issues.apache.org/jira/browse/FLINK-4478
>             Project: Flink
>          Issue Type: Improvement
>          Components: Distributed Coordination
>    Affects Versions: 1.1.0
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>             Fix For: 1.2.0
>
>
> With the Flip-6 refactoring, we'll have the need for a dedicated heartbeat component.
The heartbeat component is used to check the liveliness of the distributed components among
each other. Furthermore, heartbeats are used to regularly transmit status updates to another
component. For example, the TaskManager informs the ResourceManager with each heartbeat about
the current slot allocation.
> The heartbeat is initiated from one component. This component sends a heartbeat request
to another component which answers with an heartbeat response. Thus, one can differentiate
between a sending and a receiving side. Apart from the triggering of the heartbeat request,
the logic of treating heartbeats, marking components dead and payload delivery are the same
and should be reusable by different distributed components (JM, TM, RM).
> Different models for the heartbeat reporting are conceivable. First of all, the heartbeat
request could be sent as an ask operation where the heartbeat response is returned as a future
on the sending side. Alternatively, the sending side could request a heartbeat response by
sending a tell message. The heartbeat response is then delivered by an RPC back to the heartbeat
sender. The latter model has the advantage that a heartbeat response is not tightly coupled
to a heartbeat request. Such a tight coupling could cause that heartbeat response are ignored
after the future has timed out even though they might still contain valuable information (receiver
is still alive).
> Furthermore, different strategies for the heartbeat triggering and marking heartbeat
targets as dead are conceivable. For example, we could periodically (with a fixed period)
trigger a heartbeat request and mark all targets as dead if we didn't receive a heartbeat
response in a given time period. Furthermore, we could adapt the heartbeat interval and heartbeat
timeouts with respect to the latency of previous heartbeat responses. This would reflect the
current load and network conditions better.
> For the first version, I would propose to use a fixed period heartbeat with a maximum
heartbeat timeout before a target is marked dead. Furthermore, I would propose to use tell
messages (fire and forget) to request and report heartbeats because they are the more flexible
model imho.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message