zookeeper-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From he...@apache.org
Subject svn commit: r1368737 - in /zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/server/quorum/ src/java/test/org/apache/zookeeper/test/
Date Thu, 02 Aug 2012 22:17:59 GMT
Author: henry
Date: Thu Aug  2 22:17:58 2012
New Revision: 1368737

URL: http://svn.apache.org/viewvc?rev=1368737&view=rev
Log:
ZOOKEEPER-1514. FastLeaderElection - leader ignores the round information when joining a quorum
(flavio via henryr)

Added:
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEBackwardElectionRoundTest.java
      - copied, changed from r1368299, zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETestUtils.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1368737&r1=1368736&r2=1368737&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Thu Aug  2 22:17:58 2012
@@ -221,6 +221,9 @@ BUGFIXES:
   ZOOKEEPER-1522. intermittent failures in Zab test due to NPE in
     recursiveDelete test function (phunt via flavio)
 
+  ZOOKEEPER-1514. FastLeaderElection - leader ignores the round
+  information when joining a quorum (flavio via henryr)
+
 IMPROVEMENTS:
 
   ZOOKEEPER-1170. Fix compiler (eclipse) warnings: unused imports,

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=1368737&r1=1368736&r2=1368737&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
Thu Aug  2 22:17:58 2012
@@ -442,7 +442,7 @@ public class FastLeaderElection implemen
      * Returns the current vlue of the logical clock counter
      */
     public long getLogicalClock(){
-    return logicalclock;
+        return logicalclock;
     }
 
     /**
@@ -621,7 +621,9 @@ public class FastLeaderElection implemen
         if(leader != self.getId()){
             if(votes.get(leader) == null) predicate = false;
             else if(votes.get(leader).getState() != ServerState.LEADING) predicate = false;
-        }
+        } else if(logicalclock != electionEpoch) {
+            predicate = false;
+        } 
 
         return predicate;
     }
@@ -724,7 +726,7 @@ public class FastLeaderElection implemen
 
             synchronized(this){
                 logicalclock++;
-                    updateProposal(getInitId(), getInitLastLoggedZxid(), getPeerEpoch());
+                updateProposal(getInitId(), getInitLastLoggedZxid(), getPeerEpoch());
             }
 
             LOG.info("New election. My id =  " + self.getId() +

Copied: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEBackwardElectionRoundTest.java
(from r1368299, zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java)
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEBackwardElectionRoundTest.java?p2=zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEBackwardElectionRoundTest.java&p1=zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java&r1=1368299&r2=1368737&rev=1368737&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEBackwardElectionRoundTest.java
Thu Aug  2 22:17:58 2012
@@ -1,5 +1,4 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
+/* 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
@@ -38,16 +37,17 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-public class FLELostMessageTest extends ZKTestCase {
-    protected static final Logger LOG = LoggerFactory.getLogger(FLELostMessageTest.class);
-
+import org.apache.zookeeper.test.FLETestUtils.LEThread;
 
+public class FLEBackwardElectionRoundTest extends ZKTestCase {
+    protected static final Logger LOG = LoggerFactory.getLogger(FLELostMessageTest.class);
+    
     int count;
     HashMap<Long,QuorumServer> peers;
     File tmpdir[];
     int port[];
 
-    QuorumCnxManager cnxManager;
+    QuorumCnxManager cnxManagers[];
 
     @Before
     public void setUp() throws Exception {
@@ -56,54 +56,39 @@ public class FLELostMessageTest extends 
         peers = new HashMap<Long,QuorumServer>(count);
         tmpdir = new File[count];
         port = new int[count];
+        cnxManagers = new QuorumCnxManager[count - 1];
     }
 
     @After
     public void tearDown() throws Exception {
-        cnxManager.halt();
-    }
-
-
-    static class LEThread extends Thread {
-        private int i;
-        private QuorumPeer peer;
-
-        LEThread(QuorumPeer peer, int i) {
-            this.i = i;
-            this.peer = peer;
-            LOG.info("Constructor: " + getName());
-
-        }
-
-        public void run(){
-            try{
-                Vote v = null;
-                peer.setPeerState(ServerState.LOOKING);
-                LOG.info("Going to call leader election: " + i);
-                v = peer.getElectionAlg().lookForLeader();
-
-                if (v == null){
-                    Assert.fail("Thread " + i + " got a null vote");
-                }
-
-                /*
-                 * A real zookeeper would take care of setting the current vote. Here
-                 * we do it manually.
-                 */
-                peer.setCurrentVote(v);
-
-                LOG.info("Finished election: " + i + ", " + v.getId());
-
-                Assert.assertTrue("State is not leading.", peer.getPeerState() == ServerState.LEADING);
-            } catch (Exception e) {
-                e.printStackTrace();
+        for(int i = 0; i < (count - 1); i++){
+            if(cnxManagers[i] != null){
+                cnxManagers[i].halt();
             }
-            LOG.info("Joining");
         }
     }
+    
+    /**
+     * This test is checking the following case. A server S is
+     * currently LOOKING and it receives notifications from 
+     * a quorum indicating they are following S. The election
+     * round E of S is higher than the election round E' in the 
+     * notification messages, so S becomes the leader and sets
+     * its epoch back to E'. In the meanwhile, one or more
+     * followers turn to LOOKING and elect S in election round E.
+     * Having leader and followers with different election rounds
+     * might prevent other servers from electing a leader because
+     * they can't get a consistent set of notifications from a 
+     * quorum. 
+     * 
+     * {@link https://issues.apache.org/jira/browse/ZOOKEEPER-1514}
+     *    
+     * 
+     * @throws Exception
+     */
+    
     @Test
-    public void testLostMessage() throws Exception {
-
+    public void testBackwardElectionRound() throws Exception {
         LOG.info("TestLE: " + getTestName()+ ", " + count);
         for(int i = 0; i < count; i++) {
             int clientport = PortAssignment.unique();
@@ -119,53 +104,51 @@ public class FLELostMessageTest extends 
          * Start server 0
          */
 
-        QuorumPeer peer = new QuorumPeer(peers, tmpdir[1], tmpdir[1], port[1], 3, 1, 1000,
2, 2);
+        QuorumPeer peer = new QuorumPeer(peers, tmpdir[0], tmpdir[0], port[0], 3, 0, 1000,
2, 2);
         peer.startLeaderElection();
-        LEThread thread = new LEThread(peer, 1);
-        thread.start();
-
+        FLETestUtils.LEThread thread = new FLETestUtils.LEThread(peer, 0);
+        thread.start();  
+        
+        
         /*
          * Start mock server 1
          */
-        mockServer();
-        thread.join(5000);
-        if (thread.isAlive()) {
-            Assert.fail("Threads didn't join");
-        }
-    }
-
-    ByteBuffer createMsg(int state, long leader, long zxid, long epoch){
-        byte requestBytes[] = new byte[28];
-        ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes);
+        QuorumPeer mockPeer = new QuorumPeer(peers, tmpdir[1], tmpdir[1], port[1], 3, 1,
1000, 2, 2);
+        cnxManagers[0] = new QuorumCnxManager(mockPeer);
+        QuorumCnxManager.Listener listener = cnxManagers[0].listener;
+        listener.start();
 
+        cnxManagers[0].toSend(0l, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(),
0, 0, 1));
+        
         /*
-         * Building notification packet to send
+         * Start mock server 2
          */
+        mockPeer = new QuorumPeer(peers, tmpdir[2], tmpdir[2], port[2], 3, 2, 1000, 2, 2);
+        cnxManagers[1] = new QuorumCnxManager(mockPeer);
+        listener = cnxManagers[1].listener;
+        listener.start();
 
-        requestBuffer.clear();
-        requestBuffer.putInt(state);
-        requestBuffer.putLong(leader);
-        requestBuffer.putLong(zxid);
-        requestBuffer.putLong(epoch);
-
-        return requestBuffer;
-    }
-
-    void mockServer() throws InterruptedException, IOException {
+        cnxManagers[1].toSend(0l, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(),
0, 0, 1));
+        
         /*
-         * Create an instance of the connection manager
+         * Run another instance of leader election.
          */
-        QuorumPeer peer = new QuorumPeer(peers, tmpdir[0], tmpdir[0], port[0], 3, 0, 1000,
2, 2);
-        cnxManager = new QuorumCnxManager(peer);
-        QuorumCnxManager.Listener listener = cnxManager.listener;
-        if(listener != null){
-            listener.start();
-        } else {
-            LOG.error("Null listener when initializing cnx manager");
+        thread.join(5000);
+        thread = new FLETestUtils.LEThread(peer, 0);
+        thread.start();
+        
+        /*
+         * Send the same messages, this time should not make 0 the leader.
+         */
+        cnxManagers[0].toSend(0l, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(),
0, 0, 1));
+        cnxManagers[1].toSend(0l, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(),
0, 0, 1));
+        
+        
+        thread.join(5000);
+        
+        if (!thread.isAlive()) {
+            Assert.fail("Should not have joined");
         }
-
-        cnxManager.toSend(1l, createMsg(ServerState.LOOKING.ordinal(), 0, 0, 1));
-        cnxManager.recvQueue.take();
-        cnxManager.toSend(1L, createMsg(ServerState.FOLLOWING.ordinal(), 1, 0, 1));
+        
     }
-}
+}
\ No newline at end of file

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java?rev=1368737&r1=1368736&r2=1368737&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java Thu Aug
 2 22:17:58 2012
@@ -62,45 +62,7 @@ public class FLELostMessageTest extends 
     public void tearDown() throws Exception {
         cnxManager.halt();
     }
-
-
-    static class LEThread extends Thread {
-        private int i;
-        private QuorumPeer peer;
-
-        LEThread(QuorumPeer peer, int i) {
-            this.i = i;
-            this.peer = peer;
-            LOG.info("Constructor: " + getName());
-
-        }
-
-        public void run(){
-            try{
-                Vote v = null;
-                peer.setPeerState(ServerState.LOOKING);
-                LOG.info("Going to call leader election: " + i);
-                v = peer.getElectionAlg().lookForLeader();
-
-                if (v == null){
-                    Assert.fail("Thread " + i + " got a null vote");
-                }
-
-                /*
-                 * A real zookeeper would take care of setting the current vote. Here
-                 * we do it manually.
-                 */
-                peer.setCurrentVote(v);
-
-                LOG.info("Finished election: " + i + ", " + v.getId());
-
-                Assert.assertTrue("State is not leading.", peer.getPeerState() == ServerState.LEADING);
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-            LOG.info("Joining");
-        }
-    }
+    
     @Test
     public void testLostMessage() throws Exception {
 
@@ -121,7 +83,7 @@ public class FLELostMessageTest extends 
 
         QuorumPeer peer = new QuorumPeer(peers, tmpdir[1], tmpdir[1], port[1], 3, 1, 1000,
2, 2);
         peer.startLeaderElection();
-        LEThread thread = new LEThread(peer, 1);
+        FLETestUtils.LEThread thread = new FLETestUtils.LEThread(peer, 1);
         thread.start();
 
         /*
@@ -134,23 +96,6 @@ public class FLELostMessageTest extends 
         }
     }
 
-    ByteBuffer createMsg(int state, long leader, long zxid, long epoch){
-        byte requestBytes[] = new byte[28];
-        ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes);
-
-        /*
-         * Building notification packet to send
-         */
-
-        requestBuffer.clear();
-        requestBuffer.putInt(state);
-        requestBuffer.putLong(leader);
-        requestBuffer.putLong(zxid);
-        requestBuffer.putLong(epoch);
-
-        return requestBuffer;
-    }
-
     void mockServer() throws InterruptedException, IOException {
         /*
          * Create an instance of the connection manager
@@ -158,14 +103,11 @@ public class FLELostMessageTest extends 
         QuorumPeer peer = new QuorumPeer(peers, tmpdir[0], tmpdir[0], port[0], 3, 0, 1000,
2, 2);
         cnxManager = new QuorumCnxManager(peer);
         QuorumCnxManager.Listener listener = cnxManager.listener;
-        if(listener != null){
-            listener.start();
-        } else {
-            LOG.error("Null listener when initializing cnx manager");
-        }
+        listener.start();
+
 
-        cnxManager.toSend(1l, createMsg(ServerState.LOOKING.ordinal(), 0, 0, 1));
+        cnxManager.toSend(1l, FLETestUtils.createMsg(ServerState.LOOKING.ordinal(), 0, 0,
1));
         cnxManager.recvQueue.take();
-        cnxManager.toSend(1L, createMsg(ServerState.FOLLOWING.ordinal(), 1, 0, 1));
+        cnxManager.toSend(1L, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(), 1,
0, 1));
     }
 }

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETestUtils.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETestUtils.java?rev=1368737&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETestUtils.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETestUtils.java Thu Aug  2 22:17:58
2012
@@ -0,0 +1,98 @@
+/* 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.nio.ByteBuffer;
+
+import org.apache.zookeeper.server.quorum.QuorumPeer;
+import org.apache.zookeeper.server.quorum.Vote;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.junit.Assert;
+
+import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState;
+
+public class FLETestUtils {
+    protected static final Logger LOG = LoggerFactory.getLogger(FLETestUtils.class);
+    
+    
+    /*
+     * Thread to run an instance of leader election for 
+     * a given quorum peer.
+     */
+    static class LEThread extends Thread {
+        private int i;
+        private QuorumPeer peer;
+
+        LEThread(QuorumPeer peer, int i) {
+            this.i = i;
+            this.peer = peer;
+            LOG.info("Constructor: " + getName());
+
+        }
+
+        public void run(){
+            try{
+                Vote v = null;
+                peer.setPeerState(ServerState.LOOKING);
+                LOG.info("Going to call leader election: " + i);
+                v = peer.getElectionAlg().lookForLeader();
+
+                if (v == null){
+                    Assert.fail("Thread " + i + " got a null vote");
+                }
+
+                /*
+                 * A real zookeeper would take care of setting the current vote. Here
+                 * we do it manually.
+                 */
+                peer.setCurrentVote(v);
+
+                LOG.info("Finished election: " + i + ", " + v.getId());
+
+                Assert.assertTrue("State is not leading.", peer.getPeerState() == ServerState.LEADING);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+            LOG.info("Joining");
+        }
+    }
+    
+    /*
+     * Creates a leader election notification message.
+     */
+    
+    static ByteBuffer createMsg(int state, long leader, long zxid, long epoch){
+        byte requestBytes[] = new byte[28];
+        ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes);
+
+        /*
+         * Building notification packet to send
+         */
+
+        requestBuffer.clear();
+        requestBuffer.putInt(state);
+        requestBuffer.putLong(leader);
+        requestBuffer.putLong(zxid);
+        requestBuffer.putLong(epoch);
+
+        return requestBuffer;
+    }
+
+}
\ No newline at end of file



Mime
View raw message