You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ca...@apache.org on 2012/03/20 01:33:49 UTC

svn commit: r1302736 - in /zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java src/java/test/org/apache/zookeeper/test/FLEPredicateTest.java

Author: camille
Date: Tue Mar 20 00:33:49 2012
New Revision: 1302736

URL: http://svn.apache.org/viewvc?rev=1302736&view=rev
Log:
ZOOKEEPER-1419. Leader election never settles for a 5-node cluster (flavio via camille)
  

Added:
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEPredicateTest.java   (with props)
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1302736&r1=1302735&r2=1302736&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Tue Mar 20 00:33:49 2012
@@ -162,6 +162,8 @@ BUGFIXES:
 
   ZOOKEEPER-1384. test-cppunit overrides LD_LIBRARY_PATH and fails if
   gcc is in non-standard location (Jay Shrauner via phunt)
+  
+  ZOOKEEPER-1419. Leader election never settles for a 5-node cluster (flavio via camille)
    
 IMPROVEMENTS:
 

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java?rev=1302736&r1=1302735&r2=1302736&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java Tue Mar 20 00:33:49 2012
@@ -546,16 +546,24 @@ public class FastLeaderElection implemen
      * @param id    Server identifier
      * @param zxid  Last zxid observed by the issuer of this vote
      */
-    private boolean totalOrderPredicate(long newId, long newZxid, long newEpoch, long curId, long curZxid, long curEpoch) {
+    protected boolean totalOrderPredicate(long newId, long newZxid, long newEpoch, long curId, long curZxid, long curEpoch) {
         LOG.debug("id: " + newId + ", proposed id: " + curId + ", zxid: 0x" +
                 Long.toHexString(newZxid) + ", proposed zxid: 0x" + Long.toHexString(curZxid));
         if(self.getQuorumVerifier().getWeight(newId) == 0){
             return false;
         }
         
+        /*
+         * We return true if one of the following three cases hold:
+         * 1- New epoch is higher
+         * 2- New epoch is the same as current epoch, but new zxid is higher
+         * 3- New epoch is the same as current epoch, new zxid is the same
+         *  as current zxid, but server id is higher.
+         */
+        
         return ((newEpoch > curEpoch) || 
-                ((newEpoch == curEpoch) && (newZxid > curZxid)) || 
-                ((newZxid == curZxid) && (newId > curId)));
+                ((newEpoch == curEpoch) &&
+                ((newZxid > curZxid) || ((newZxid == curZxid) && (newId > curId)))));
     }
 
     /**

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEPredicateTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEPredicateTest.java?rev=1302736&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEPredicateTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEPredicateTest.java Tue Mar 20 00:33:49 2012
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.test;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+
+import org.apache.zookeeper.server.quorum.FastLeaderElection;
+import org.apache.zookeeper.server.quorum.QuorumCnxManager;
+import org.apache.zookeeper.server.quorum.QuorumPeer;
+import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZKTestCase;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class FLEPredicateTest extends ZKTestCase {
+    
+    protected static final Logger LOG = LoggerFactory.getLogger(FLEPredicateTest.class);
+    
+    class MockFLE extends FastLeaderElection {
+        MockFLE(QuorumPeer peer){
+            super(peer, new QuorumCnxManager(peer));
+        }
+        
+        boolean predicate(long newId, long newZxid, long newEpoch, long curId, long curZxid, long curEpoch){
+            return this.totalOrderPredicate(newId, newZxid, newEpoch, curId, curZxid, curEpoch);
+        }
+    }
+    
+    
+    HashMap<Long,QuorumServer> peers;
+    
+    @Test
+    public void testPredicate() throws IOException {
+        
+        peers = new HashMap<Long,QuorumServer>(3);
+        
+        /*
+         * Creates list of peers.
+         */
+        for(int i = 0; i < 3; i++) {
+            peers.put(Long.valueOf(i),
+                    new QuorumServer(i,
+                            new InetSocketAddress(PortAssignment.unique()),
+                    new InetSocketAddress(PortAssignment.unique())));
+        }
+
+        /*
+         * Creating peer.
+         */
+        try{
+            File tmpDir = ClientBase.createTmpDir();
+            QuorumPeer peer = new QuorumPeer(peers, tmpDir, tmpDir,
+                                        PortAssignment.unique(), 3, 0, 1000, 2, 2);
+        
+            MockFLE mock = new MockFLE(peer);
+            
+            /*
+             * Lower epoch must return false
+             */
+            
+            Assert.assertFalse (mock.predicate(4L, 0L, 0L, 3L, 0L, 2L));
+            
+            /*
+             * Later epoch
+             */
+            Assert.assertTrue (mock.predicate(0L, 0L, 1L, 1L, 0L, 0L));
+        
+            /*
+             * Higher zxid
+             */
+            Assert.assertTrue(mock.predicate(0L, 1L, 0L, 1L, 0L, 0L));
+        
+            /*
+             * Higher id
+             */
+            Assert.assertTrue(mock.predicate(1L, 1L, 0L, 0L, 1L, 0L));
+        } catch (IOException e) {
+            LOG.error("Exception while creating quorum peer", e);
+            Assert.fail("Exception while creating quorum peer");
+        }
+    }
+}

Propchange: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEPredicateTest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain