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

ASF GitHub Bot commented on TWILL-129:
--------------------------------------

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

    https://github.com/apache/incubator-twill/pull/34#discussion_r30920964
  
    --- Diff: 
twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/LeaderElection.java
 ---
    @@ -240,10 +240,10 @@ private void becomeFollower() {
        * Starts watching for the max. of smaller node.
        */
       private void watchNode(final String nodePath, Watcher watcher) {
    -    OperationFuture<Stat> watchFuture = zkClient.exists(nodePath, watcher);
    -    Futures.addCallback(watchFuture, new FutureCallback<Stat>() {
    +    OperationFuture<NodeData> watchFuture = zkClient.getData(nodePath, 
watcher);
    --- End diff --
    
    So this is relying on the fact that a KeeperException.NoNodeException will 
be thrown if the node has already been removed?  That would trigger the 
onFailure() hook and re-run the election.  Is that right?


> Race condition in leader election if leader quitted before follower set the 
> watch
> ---------------------------------------------------------------------------------
>
>                 Key: TWILL-129
>                 URL: https://issues.apache.org/jira/browse/TWILL-129
>             Project: Apache Twill
>          Issue Type: Bug
>          Components: zookeeper
>    Affects Versions: 0.4.0-incubating, 0.5.0-incubating, 0.4.1-incubating
>            Reporter: Terence Yim
>            Assignee: Terence Yim
>            Priority: Blocker
>
> In the LeaderElection implementation, if the leader quit and delete its node 
> before the next-in-line follower try to set the watch through the exists() 
> call, the follower will fail silently without noticing the leader node is 
> gone.



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

Reply via email to