You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@twill.apache.org by "Terence Yim (JIRA)" <ji...@apache.org> on 2015/05/22 08:03:17 UTC
[jira] [Commented] (TWILL-129) Race condition in leader election if
leader quitted before follower set the watch
[ https://issues.apache.org/jira/browse/TWILL-129?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14555632#comment-14555632 ]
Terence Yim commented on TWILL-129:
-----------------------------------
This is the unit-test to reproduce the problem.
{code:java}
@Test
public void testRace() throws InterruptedException {
ExecutorService executor = Executors.newFixedThreadPool(2);
final AtomicInteger leaderCount = new AtomicInteger(0);
final CountDownLatch completeLatch = new CountDownLatch(2);
// Starts two threads and try to compete for leader and immediate drop leadership.
// This is to test the case when a follower tries to watch for leader node, but the leader is already gone
for (int i = 0; i < 2; i++) {
final ZKClientService zkClient = ZKClientService.Builder.of(zkServer.getConnectionStr()).build();
zkClient.startAndWait();
executor.execute(new Runnable() {
@Override
public void run() {
try {
for (int i = 0; i < 1000; i++) {
final CountDownLatch leaderLatch = new CountDownLatch(1);
LeaderElection election = new LeaderElection(zkClient, "/testRace", new ElectionHandler() {
@Override
public void leader() {
leaderCount.incrementAndGet();
leaderLatch.countDown();
}
@Override
public void follower() {
// no-op
}
});
election.startAndWait();
Uninterruptibles.awaitUninterruptibly(leaderLatch);
election.stopAndWait();
}
completeLatch.countDown();
} finally {
zkClient.stopAndWait();
}
}
});
}
try {
Assert.assertTrue(completeLatch.await(2, TimeUnit.MINUTES));
} finally {
executor.shutdownNow();
}
}
{code}
> 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)