You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/07/28 16:10:01 UTC

[jira] [Commented] (FLINK-7295) Add callback for proper RpcEndpoint shut down

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

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

GitHub user tillrohrmann opened a pull request:

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

    [FLINK-7295] [rpc] Add postStop callback for proper shutdown of RpcEndpoints

    ## What is the purpose of the change
    
    In order to execute a proper shutdown of RpcEndpoints it is necessary to have
    a callback which is executed in the main thread context directly before stopping
    processing of messages. This PR introduces the postStop method which acts as
    this callback. All endpoint specific cleanup should be executed in this method.
    
    The RpcEndpoint#shutDown method now only triggers the shut down of an RpcEndpoint.
    In order to wait on the completion of the shut down, one has to wait on the
    termination future which can be retrieved via RpcEndpoint#getTerminationFuture.
    
    This PR also adapts the existing RpcEndpoints such that they execute their former
    shutDown logic in the postStop method.
    
    ## Brief change log
    
      - Introduced `AkkaRpcService` specific `Shutdown` message
      - made `RpcEndpoint#shutDown` final
      - introduced `RpcEndpoint#postStop` call back
      - Adapted existing `RpcEndpoints` accordingly
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
    
    - Added test that `postStop` is executed in the main thread
    - Added test that `Exceptions` occurring in the `postStop` method are returned as the result of the termination future
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (no)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes) Flip-6
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
    


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

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

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

    https://github.com/apache/flink/pull/4420.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 #4420
    
----
commit a23aba69e365fa87204b894640452afb097f5bb6
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-07-28T14:13:55Z

    [FLINK-7295] [rpc] Add postStop callback for proper shutdown of RpcEndpoints
    
    In order to execute a proper shutdown of RpcEndpoints it is necessary to have
    a callback which is executed in the main thread context directly before stopping
    processing of messages. This PR introduces the postStop method which acts as
    this callback. All endpoint specific cleanup should be executed in this method.
    
    The RpcEndpoint#shutDown method now only triggers the shut down of an RpcEndpoint.
    In order to wait on the completion of the shut down, one has to wait on the
    termination future which can be retrieved via RpcEndpoint#getTerminationFuture.
    
    This PR also adapts the existing RpcEndpoints such that they execute their former
    shutDown logic in the postStop method.

----


> Add callback for proper RpcEndpoint shut down
> ---------------------------------------------
>
>                 Key: FLINK-7295
>                 URL: https://issues.apache.org/jira/browse/FLINK-7295
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Distributed Coordination
>    Affects Versions: 1.4.0
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>
> In order to properly shut down {{RpcEndpoints}} it is necessary to have a method which is called by the main thread in case of a shut down and allows to properly close and clean up internal state. At the moment, this clean up work is done by overriding the {{RpcEndpoint#shutDown}} method which can be called by a different thread than the main thread. This is problematic since it violates the {{RpcEndpoint}} contract.
> I propose to change the behaviour of {{RpcEndpoint#shutDown}} to be asynchronous. Calling this method will send a message to the {{RpcEndpoint}} which triggers the call of the clean up method and the termination of the endpoint.
> In order to obtain the same behaviour as before, the user can obtain the termination future on which it can wait after sending the request to shut down the {{RpcEndpoint}}.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)