You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by mi...@apache.org on 2014/03/30 00:01:40 UTC

svn commit: r1583083 - in /zookeeper/trunk: CHANGES.txt src/recipes/election/src/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupport.java src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java

Author: michim
Date: Sat Mar 29 23:01:40 2014
New Revision: 1583083

URL: http://svn.apache.org/r1583083
Log:
ZOOKEEPER-1219. LeaderElectionSupport recipe is unnecessarily dispatching the READY_START event even if the ELECTED node stopped/expired simultaneously. (Rakesh R via michim)

Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/recipes/election/src/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupport.java
    zookeeper/trunk/src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1583083&r1=1583082&r2=1583083&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Sat Mar 29 23:01:40 2014
@@ -819,6 +819,10 @@ IMPROVEMENTS:
 
   ZOOKEEPER-1408. CLI: sort output of ls command (Hartmut Lang via michim)
 
+  ZOOKEEPER-1219. LeaderElectionSupport recipe is unnecessarily dispatching the
+  READY_START event even if the ELECTED node stopped/expired simultaneously.
+  (Rakesh R via michim)
+
 headers
 
 Release 3.4.0 - 

Modified: zookeeper/trunk/src/recipes/election/src/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupport.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/recipes/election/src/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupport.java?rev=1583083&r1=1583082&r2=1583083&view=diff
==============================================================================
--- zookeeper/trunk/src/recipes/election/src/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupport.java (original)
+++ zookeeper/trunk/src/recipes/election/src/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupport.java Sat Mar 29 23:01:40 2014
@@ -235,7 +235,6 @@ public class LeaderElectionSupport imple
 
   private void becomeReady(LeaderOffer neighborLeaderOffer)
       throws KeeperException, InterruptedException {
-    dispatchEvent(EventType.READY_START);
 
     logger.info("{} not elected leader. Watching node:{}",
         leaderOffer.getNodePath(), neighborLeaderOffer.getNodePath());
@@ -247,6 +246,7 @@ public class LeaderElectionSupport imple
     Stat stat = zooKeeper.exists(neighborLeaderOffer.getNodePath(), this);
 
     if (stat != null) {
+      dispatchEvent(EventType.READY_START);
       logger.debug(
           "We're behind {} in line and they're alive. Keeping an eye on them.",
           neighborLeaderOffer.getNodePath());

Modified: zookeeper/trunk/src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java?rev=1583083&r1=1583082&r2=1583083&view=diff
==============================================================================
--- zookeeper/trunk/src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java (original)
+++ zookeeper/trunk/src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java Sat Mar 29 23:01:40 2014
@@ -29,6 +29,7 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.recipes.leader.LeaderElectionSupport.EventType;
 import org.apache.zookeeper.test.ClientBase;
 import org.junit.After;
 import org.junit.Before;
@@ -202,6 +203,54 @@ public class LeaderElectionSupportTest e
     electionSupport.stop();
   }
 
+  @Test
+  public void testReadyOffer() throws Exception {
+      final ArrayList<EventType> events = new ArrayList<EventType>();
+      final CountDownLatch electedComplete = new CountDownLatch(1);
+
+      final LeaderElectionSupport electionSupport1 = createLeaderElectionSupport();
+      electionSupport1.start();
+      LeaderElectionSupport electionSupport2 = createLeaderElectionSupport();
+      LeaderElectionAware listener = new LeaderElectionAware() {
+          boolean stoppedElectedNode = false;
+          @Override
+          public void onElectionEvent(EventType eventType) {
+              events.add(eventType);
+              if (!stoppedElectedNode
+                      && eventType == EventType.DETERMINE_COMPLETE) {
+                  stoppedElectedNode = true;
+                  try {
+                      // stopping the ELECTED node, so re-election will happen.
+                      electionSupport1.stop();
+                  } catch (Exception e) {
+                      logger.error("Unexpected error", e);
+                  }
+              }
+              if (eventType == EventType.ELECTED_COMPLETE) {
+                  electedComplete.countDown();
+              }
+          }
+      };
+      electionSupport2.addListener(listener);
+      electionSupport2.start();
+      // waiting for re-election.
+      electedComplete.await(CONNECTION_TIMEOUT / 3, TimeUnit.MILLISECONDS);
+
+      final ArrayList<EventType> expectedevents = new ArrayList<EventType>();
+      expectedevents.add(EventType.START);
+      expectedevents.add(EventType.OFFER_START);
+      expectedevents.add(EventType.OFFER_COMPLETE);
+      expectedevents.add(EventType.DETERMINE_START);
+      expectedevents.add(EventType.DETERMINE_COMPLETE);
+      expectedevents.add(EventType.DETERMINE_START);
+      expectedevents.add(EventType.DETERMINE_COMPLETE);
+      expectedevents.add(EventType.ELECTED_START);
+      expectedevents.add(EventType.ELECTED_COMPLETE);
+      Assert.assertEquals("Events has failed to executed in the order",
+              expectedevents, events);
+      electionSupport2.stop();
+  }
+  
   private LeaderElectionSupport createLeaderElectionSupport() {
     LeaderElectionSupport electionSupport = new LeaderElectionSupport();