Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id B5081200BF8 for ; Fri, 13 Jan 2017 22:27:30 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id B3A5B160B3F; Fri, 13 Jan 2017 21:27:30 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 097FE160B2E for ; Fri, 13 Jan 2017 22:27:29 +0100 (CET) Received: (qmail 18929 invoked by uid 500); 13 Jan 2017 21:27:29 -0000 Mailing-List: contact dev-help@curator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@curator.apache.org Delivered-To: mailing list dev@curator.apache.org Received: (qmail 18918 invoked by uid 99); 13 Jan 2017 21:27:29 -0000 Received: from Unknown (HELO jira-lw-us.apache.org) (207.244.88.139) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 13 Jan 2017 21:27:29 +0000 Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 8AE3E2528A for ; Fri, 13 Jan 2017 21:27:26 +0000 (UTC) Date: Fri, 13 Jan 2017 21:27:26 +0000 (UTC) From: "Jordan Zimmerman (JIRA)" To: dev@curator.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (CURATOR-171) LeaderLatch isn't aware if it's own ephemeral node goes away MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 13 Jan 2017 21:27:30 -0000 [ https://issues.apache.org/jira/browse/CURATOR-171?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15822373#comment-15822373 ] Jordan Zimmerman commented on CURATOR-171: ------------------------------------------ The leader latch has never watched its own node. That's not in its contract. Further, there's overhead to doing this. None of the Curator lock/leader recipes watch their own nodes. There's no reason to. Please see Tech Note 7: https://cwiki.apache.org/confluence/display/CURATOR/TN7 > LeaderLatch isn't aware if it's own ephemeral node goes away > ------------------------------------------------------------ > > Key: CURATOR-171 > URL: https://issues.apache.org/jira/browse/CURATOR-171 > Project: Apache Curator > Issue Type: Bug > Components: Recipes > Reporter: John Vines > > Running the following code- {code} public static void main(String args[]) throws Exception { > CuratorFramework curator = CuratorFrameworkFactory.builder().connectString("localhost:2181").retryPolicy(new ExponentialBackoffRetry(1000, 5)) > .authorization("digest", "accumulo:DEFAULT".getBytes()).build(); > curator.start(); > LeaderLatch latch = new LeaderLatch(curator, "/latch", "test"); > LeaderLatch latch2 = new LeaderLatch(curator, "/latch", "test2"); > latch.addListener(new LeaderLatchListener() { > @Override > public void isLeader() { > System.out.println("Became leader!"); > } > @Override > public void notLeader() { > System.out.println("Lost leadership!"); > } > }); > latch.start(); > latch.await(); > latch2.start(); > Thread.sleep(1000); > System.out.println("Does latch1 have leadership? " + latch.hasLeadership()); > System.out.println("Does latch2 have leadership? " + latch2.hasLeadership()); > for (String child : curator.getChildren().forPath("/latch")) > if (Arrays.equals(curator.getData().forPath(ZKPaths.makePath("/latch", child)), "test".getBytes())) > curator.delete().deletingChildrenIfNeeded().forPath(ZKPaths.makePath("/latch", child)); > Thread.sleep(1000); > System.out.println("Does latch1 have leadership? " + latch.hasLeadership()); > System.out.println("Does latch2 have leadership? " + latch2.hasLeadership()); > > Thread.sleep(1000 * 40); > System.out.println("Does latch1 have leadership? " + latch.hasLeadership()); > System.out.println("Does latch2 have leadership? " + latch2.hasLeadership()); > > latch.close(); > latch2.close(); > curator.close(); > }{code} > I get the following output-{noformat}Became leader! > Does latch1 have leadership? true > Does latch2 have leadership? false > Does latch1 have leadership? true > Does latch2 have leadership? true > Does latch1 have leadership? true > Does latch2 have leadership? true > {noformat} > I expect that when the ephemeral node is deleted, latch1 no longer reports itself as having leadership. Furthermore, I expect it to trigger the notLeader call. -- This message was sent by Atlassian JIRA (v6.3.4#6332)