You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by br...@apache.org on 2011/06/17 01:34:31 UTC

svn commit: r1136740 - in /zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/server/quorum/ src/java/test/org/apache/zookeeper/test/

Author: breed
Date: Thu Jun 16 23:34:30 2011
New Revision: 1136740

URL: http://svn.apache.org/viewvc?rev=1136740&view=rev
Log:
ZOOKEEPER-1094. Small improvements to LeaderElection and Vote classes

Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Learner.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Vote.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Thu Jun 16 23:34:30 2011
@@ -308,6 +308,8 @@ IMPROVEMENTS:
 
   ZOOKEEPER-1052. Findbugs warning in QuorumPeer.ResponderThread.run()  (fpj via michim)
 
+  ZOOKEEPER-1094. Small improvements to LeaderElection and Vote classes (henryr via breed)
+
 NEW FEATURES:
   ZOOKEEPER-729. Java client API to recursively delete a subtree.
   (Kay Kay via henry)

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java Thu Jun 16 23:34:30 2011
@@ -279,7 +279,7 @@ public class AuthFastLeaderElection impl
                     case 0:
                         // Receive challenge request
                         ToSend c = new ToSend(ToSend.mType.challenge, tag,
-                                current.id, current.zxid,
+                                current.getId(), current.getZxid(),
                                 logicalclock, self.getPeerState(),
                                 (InetSocketAddress) responsePacket
                                         .getSocketAddress());
@@ -322,8 +322,8 @@ public class AuthFastLeaderElection impl
                                         recvqueue.offer(n);
 
                                         ToSend a = new ToSend(ToSend.mType.ack,
-                                                tag, current.id,
-                                                current.zxid,
+                                                tag, current.getId(),
+                                                current.getZxid(),
                                                 logicalclock, self.getPeerState(),
                                                 addr);
 
@@ -342,7 +342,7 @@ public class AuthFastLeaderElection impl
                             recvqueue.offer(n);
 
                             ToSend a = new ToSend(ToSend.mType.ack, tag,
-                                    current.id, current.zxid,
+                                    current.getId(), current.getZxid(),
                                     logicalclock, self.getPeerState(),
                                     (InetSocketAddress) responsePacket
                                             .getSocketAddress());
@@ -804,7 +804,7 @@ public class AuthFastLeaderElection impl
          * zxids for a server depending on timing.
          */
         for (Vote v : votesCast) {
-            if ((v.id == l) && (v.zxid == zxid))
+            if ((v.getId() == l) && (v.getZxid() == zxid))
                 count++;
         }
 

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=1136740&r1=1136739&r2=1136740&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 Jun 16 23:34:30 2011
@@ -216,12 +216,12 @@ public class FastLeaderElection implemen
                         if(!self.getVotingView().containsKey(response.sid)){
                             Vote current = self.getCurrentVote();
                             ToSend notmsg = new ToSend(ToSend.mType.notification,
-                                    current.id,
-                                    current.zxid,
+                                    current.getId(),
+                                    current.getZxid(),
                                     logicalclock,
                                     self.getPeerState(),
                                     response.sid,
-                                    current.peerEpoch);
+                                    current.getPeerEpoch());
 
                             sendqueue.offer(notmsg);
                         } else {
@@ -298,12 +298,12 @@ public class FastLeaderElection implemen
                                         && (n.electionEpoch < logicalclock)){
                                     Vote v = getVote();
                                     ToSend notmsg = new ToSend(ToSend.mType.notification,
-                                            v.id,
-                                            v.zxid,
+                                            v.getId(),
+                                            v.getZxid(),
                                             logicalclock,
                                             self.getPeerState(),
                                             response.sid,
-                                            v.peerEpoch);
+                                            v.getPeerEpoch());
                                     sendqueue.offer(notmsg);
                                 }
                             } else {
@@ -317,17 +317,17 @@ public class FastLeaderElection implemen
                                         LOG.debug("Sending new notification. My id =  " +
                                                 self.getId() + ", Recipient = " +
                                                 response.sid + " zxid =" +
-                                                current.zxid + " leader=" +
-                                                current.id);
+                                                current.getZxid() + " leader=" +
+                                                current.getId());
                                     }
                                     ToSend notmsg = new ToSend(
                                             ToSend.mType.notification,
-                                            current.id,
-                                            current.zxid,
+                                            current.getId(),
+                                            current.getZxid(),
                                             logicalclock,
                                             self.getPeerState(),
                                             response.sid,
-                                            current.peerEpoch);
+                                            current.getPeerEpoch());
                                     sendqueue.offer(notmsg);
                                 }
                             }
@@ -491,8 +491,8 @@ public class FastLeaderElection implemen
     private void leaveInstance(Vote v) {
         if(LOG.isDebugEnabled()){
             LOG.debug("About to leave FLE instance: Leader= "
-                + v.id + ", Zxid = " +
-                v.zxid + ", My id = " + self.getId()
+                + v.getId() + ", Zxid = " +
+                v.getZxid() + ", My id = " + self.getId()
                 + ", My state = " + self.getPeerState());
         }
         recvqueue.clear();
@@ -618,7 +618,7 @@ public class FastLeaderElection implemen
 
         if(leader != self.getId()){
             if(votes.get(leader) == null) predicate = false;
-            else if(votes.get(leader).state != ServerState.LEADING) predicate = false;
+            else if(votes.get(leader).getState() != ServerState.LEADING) predicate = false;
         }
 
         return predicate;

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java Thu Jun 16 23:34:30 2011
@@ -28,6 +28,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.Map;
 import java.util.Random;
 import java.util.Map.Entry;
 
@@ -50,7 +51,7 @@ public class LeaderElection implements E
         this.self = self;
     }
 
-    public static class ElectionResult {
+    protected static class ElectionResult {
         public Vote vote;
 
         public int count;
@@ -58,44 +59,59 @@ public class LeaderElection implements E
         public Vote winner;
 
         public int winningCount;
+
+        public int numValidVotes;
     }
 
     protected ElectionResult countVotes(HashMap<InetSocketAddress, Vote> votes, HashSet<Long> heardFrom) {
-        ElectionResult result = new ElectionResult();
+        final ElectionResult result = new ElectionResult();
         // Initialize with null vote
         result.vote = new Vote(Long.MIN_VALUE, Long.MIN_VALUE);
         result.winner = new Vote(Long.MIN_VALUE, Long.MIN_VALUE);
-        Collection<Vote> votesCast = votes.values();
-        // First make the views consistent. Sometimes peers will have
+
+        // First, filter out votes from unheard-from machines. Then
+        // make the views consistent. Sometimes peers will have
         // different zxids for a server depending on timing.
-        for (Iterator<Vote> i = votesCast.iterator(); i.hasNext();) {
-            Vote v = i.next();
-            if (!heardFrom.contains(v.id)) {
-                // Discard votes for machines that we didn't hear from
-                i.remove();
-                continue;
-            }
-            for (Vote w : votesCast) {
-                if (v.id == w.id) {
-                    if (v.zxid < w.zxid) {
-                        v.zxid = w.zxid;
+        final HashMap<InetSocketAddress, Vote> validVotes = new HashMap<InetSocketAddress, Vote>();
+        final Map<Long, Long> maxZxids = new HashMap<Long,Long>();
+        for (Map.Entry<InetSocketAddress, Vote> e : votes.entrySet()) {
+            // Only include votes from machines that we heard from
+            final Vote v = e.getValue();
+            if (heardFrom.contains(v.getId())) {
+                validVotes.put(e.getKey(), v);
+                Long val = maxZxids.get(v.getId());
+                if (val == null || val < v.getZxid()) {
+                    maxZxids.put(v.getId(), v.getZxid());
+            }
                     }
                 }
+
+        // Make all zxids for a given vote id equal to the largest zxid seen for
+        // that id
+        for (Map.Entry<InetSocketAddress, Vote> e : validVotes.entrySet()) {
+            final Vote v = e.getValue();
+            Long zxid = maxZxids.get(v.getId());
+            if (v.getZxid() < zxid) {
+                // This is safe inside an iterator as per
+                // http://download.oracle.com/javase/1.5.0/docs/api/java/util/Map.Entry.html
+                e.setValue(new Vote(v.getId(), zxid, v.getElectionEpoch(), v.getPeerEpoch(), v.getState()));
             }
         }
 
-        HashMap<Vote, Integer> countTable = new HashMap<Vote, Integer>();
+        result.numValidVotes = validVotes.size();
+
+        final HashMap<Vote, Integer> countTable = new HashMap<Vote, Integer>();
         // Now do the tally
-        for (Vote v : votesCast) {
+        for (Vote v : validVotes.values()) {
             Integer count = countTable.get(v);
             if (count == null) {
                 count = Integer.valueOf(0);
             }
             countTable.put(v, count + 1);
-            if (v.id == result.vote.id) {
+            if (v.getId() == result.vote.getId()) {
                 result.count++;
-            } else if (v.zxid > result.vote.zxid
-                    || (v.zxid == result.vote.zxid && v.id > result.vote.id)) {
+            } else if (v.getZxid() > result.vote.getZxid()
+                    || (v.getZxid() == result.vote.getZxid() && v.getId() > result.vote.getId())) {
                 result.vote = v;
                 result.count = 1;
             }
@@ -107,7 +123,7 @@ public class LeaderElection implements E
                 result.winningCount = entry.getValue();
                 result.winner = entry.getKey();
             }
-            LOG.info(entry.getKey().id + "\t-> " + entry.getValue());
+            LOG.info(entry.getKey().getId() + "\t-> " + entry.getValue());
         }
         return result;
     }
@@ -154,11 +170,11 @@ public class LeaderElection implements E
                     requestBytes.length);
             DatagramPacket responsePacket = new DatagramPacket(responseBytes,
                     responseBytes.length);
-            HashMap<InetSocketAddress, Vote> votes =
-                new HashMap<InetSocketAddress, Vote>(self.getVotingView().size());
             int xid = epochGen.nextInt();
             while (self.isRunning()) {
-                votes.clear();
+                HashMap<InetSocketAddress, Vote> votes =
+                    new HashMap<InetSocketAddress, Vote>(self.getVotingView().size());
+
                 requestBuffer.clear();
                 requestBuffer.putInt(xid);
                 requestPacket.setLength(4);
@@ -216,11 +232,11 @@ public class LeaderElection implements E
                 // If no votes are received for live peers, reset to voting 
                 // for ourselves as otherwise we may hang on to a vote 
                 // for a dead peer                 
-                if (votes.size() == 0) {                    
+                if (result.numValidVotes == 0) {
                     self.setCurrentVote(new Vote(self.getId(),
                             self.getLastLoggedZxid()));
                 } else {
-                    if (result.winner.id >= 0) {
+                    if (result.winner.getId() >= 0) {
                         self.setCurrentVote(result.vote);
                         // To do: this doesn't use a quorum verifier
                         if (result.winningCount > (self.getVotingView().size() / 2)) {
@@ -236,7 +252,7 @@ public class LeaderElection implements E
                              * error to be elected as a Leader.
                              */
                             if (self.getLearnerType() == LearnerType.OBSERVER) {
-                                if (current.id == self.getId()) {
+                                if (current.getId() == self.getId()) {
                                     // This should never happen!
                                     LOG.error("OBSERVER elected as leader!");
                                     Thread.sleep(100);
@@ -247,7 +263,7 @@ public class LeaderElection implements E
                                     return current;
                                 }
                             } else {
-                                self.setPeerState((current.id == self.getId())
+                                self.setPeerState((current.getId() == self.getId())
                                         ? ServerState.LEADING: ServerState.FOLLOWING);
                                 if (self.getPeerState() == ServerState.FOLLOWING) {
                                     Thread.sleep(100);

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Learner.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Learner.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Learner.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Learner.java Thu Jun 16 23:34:30 2011
@@ -195,14 +195,14 @@ public class Learner {       
         // Find the leader by id
         Vote current = self.getCurrentVote();
         for (QuorumServer s : self.getView().values()) {
-            if (s.id == current.id) {
+            if (s.id == current.getId()) {
                 addr = s.addr;
                 break;
             }
         }
         if (addr == null) {
             LOG.warn("Couldn't find the leader with id = "
-                    + current.id);
+                    + current.getId());
         }
         return addr;
     }

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java Thu Jun 16 23:34:30 2011
@@ -285,8 +285,8 @@ public class QuorumPeer extends Thread i
                         Vote current = getCurrentVote();
                         switch (getPeerState()) {
                         case LOOKING:
-                            responseBuffer.putLong(current.id);
-                            responseBuffer.putLong(current.zxid);
+                            responseBuffer.putLong(current.getId());
+                            responseBuffer.putLong(current.getZxid());
                             break;
                         case LEADING:
                             responseBuffer.putLong(myid);
@@ -302,7 +302,7 @@ public class QuorumPeer extends Thread i
                             }
                             break;
                         case FOLLOWING:
-                            responseBuffer.putLong(current.id);
+                            responseBuffer.putLong(current.getId());
                             try {
                                 responseBuffer.putLong(follower.getZxid());
                             } catch (NullPointerException npe) {
@@ -833,7 +833,7 @@ public class QuorumPeer extends Thread i
             if (leader != null) {
                 synchronized (leader.learners) {
                     for (LearnerHandler fh :
-                        (Collection<LearnerHandler>)leader.learners)
+                        leader.learners)
                     {
                         if (fh.getSocket() == null)
                             continue;

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Vote.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Vote.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Vote.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Vote.java Thu Jun 16 23:34:30 2011
@@ -26,12 +26,17 @@ public class Vote {
     public Vote(long id, long zxid) {
         this.id = id;
         this.zxid = zxid;
+        this.electionEpoch = -1;
+        this.peerEpoch = -1;
+        this.state = ServerState.LOOKING;
     }
     
     public Vote(long id, long zxid, long peerEpoch) {
         this.id = id;
         this.zxid = zxid;
+        this.electionEpoch = -1;
         this.peerEpoch = peerEpoch;
+        this.state = ServerState.LOOKING;
     }
 
     public Vote(long id, long zxid, long electionEpoch, long peerEpoch) {
@@ -39,6 +44,7 @@ public class Vote {
         this.zxid = zxid;
         this.electionEpoch = electionEpoch;
         this.peerEpoch = peerEpoch;
+        this.state = ServerState.LOOKING;
     }
     
     public Vote(long id, long zxid, long electionEpoch, long peerEpoch, ServerState state) {
@@ -49,15 +55,35 @@ public class Vote {
         this.peerEpoch = peerEpoch;
     }
     
-    public long id;
+    final private long id;
     
-    public long zxid;
+    final private long zxid;
     
-    public long electionEpoch = -1;
+    final private long electionEpoch;
     
-    public long peerEpoch = -1;
+    final private long peerEpoch;
     
-    public ServerState state = ServerState.LOOKING;
+    public long getId() {
+        return id;
+    }
+
+    public long getZxid() {
+        return zxid;
+    }
+
+    public long getElectionEpoch() {
+        return electionEpoch;
+    }
+
+    public long getPeerEpoch() {
+        return peerEpoch;
+    }
+
+    public ServerState getState() {
+        return state;
+    }
+
+    final private ServerState state;
     
     @Override
     public boolean equals(Object o) {

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=1136740&r1=1136739&r2=1136740&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 Jun 16 23:34:30 2011
@@ -94,7 +94,7 @@ public class FLELostMessageTest extends 
                  */
                 peer.setCurrentVote(v);
 
-                LOG.info("Finished election: " + i + ", " + v.id);
+                LOG.info("Finished election: " + i + ", " + v.getId());
                     
                 Assert.assertTrue("State is not leading.", peer.getPeerState() == ServerState.LEADING);
             } catch (Exception e) {

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java Thu Jun 16 23:34:30 2011
@@ -108,7 +108,7 @@ public class FLENewEpochTest extends ZKT
                      */
                     peer.setCurrentVote(v);
 
-                    LOG.info("Finished election: " + i + ", " + v.id);
+                    LOG.info("Finished election: " + i + ", " + v.getId());
                     //votes[i] = v;
 
                     switch (i) {

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java Thu Jun 16 23:34:30 2011
@@ -126,7 +126,7 @@ public class FLERestartTest extends ZKTe
                      */
                     peer.setCurrentVote(v);
 
-                    LOG.info("Finished election: " + i + ", " + v.id);
+                    LOG.info("Finished election: " + i + ", " + v.getId());
                     //votes[i] = v;
 
                     switch(i){

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java Thu Jun 16 23:34:30 2011
@@ -125,7 +125,7 @@ public class FLETest extends ZKTestCase 
                      */
                     peer.setCurrentVote(v);
 
-                    LOG.info("Finished election: " + i + ", " + v.id);
+                    LOG.info("Finished election: " + i + ", " + v.getId());
                     votes[i] = v;
 
                     /*
@@ -133,7 +133,7 @@ public class FLETest extends ZKTestCase 
                      */
                     int lc = (int) ((FastLeaderElection) peer.getElectionAlg()).getLogicalClock();
 
-                    if (v.id == ((long) i)) {
+                    if (v.getId() == i) {
                         /*
                          * A leader executes this part of the code. If it is the first leader to be
                          * elected, then it Assert.fails right after. Otherwise, it waits until it has enough
@@ -158,16 +158,16 @@ public class FLETest extends ZKTestCase 
                                     if(voteMap.get(lc) == null)
                                         voteMap.put(lc, new HashSet<TestVote>());
                                     HashSet<TestVote> hs = voteMap.get(lc);
-                                    hs.add(new TestVote(i, v.id));
+                                    hs.add(new TestVote(i, v.getId()));
 
-                                    if(countVotes(hs, v.id) > (count/2)){
+                                    if(countVotes(hs, v.getId()) > (count/2)){
                                         leader = i;
                                         LOG.info("Got majority: " + i);
                                     } else {
                                         voteMap.wait(3000);
                                         LOG.info("Notified or expired: " + i);
                                         hs = voteMap.get(lc);
-                                        if(countVotes(hs, v.id) > (count/2)){
+                                        if(countVotes(hs, v.getId()) > (count/2)){
                                             leader = i;
                                             LOG.info("Got majority: " + i);
                                         } else {
@@ -201,13 +201,13 @@ public class FLETest extends ZKTestCase 
 
                         LOG.info("Logical clock " + ((FastLeaderElection) peer.getElectionAlg()).getLogicalClock());
                         synchronized(voteMap){
-                            LOG.info("Voting on " + votes[i].id + ", round " + ((FastLeaderElection) peer.getElectionAlg()).getLogicalClock());
+                            LOG.info("Voting on " + votes[i].getId() + ", round " + ((FastLeaderElection) peer.getElectionAlg()).getLogicalClock());
                             if(voteMap.get(lc) == null)
                                 voteMap.put(lc, new HashSet<TestVote>());
                             HashSet<TestVote> hs = voteMap.get(lc);
-                            hs.add(new TestVote(i, votes[i].id));
-                            if(countVotes(hs, votes[i].id) > (count/2)){
-                                LOG.info("Logical clock: " + lc + ", " + votes[i].id);
+                            hs.add(new TestVote(i, votes[i].getId()));
+                            if(countVotes(hs, votes[i].getId()) > (count/2)){
+                                LOG.info("Logical clock: " + lc + ", " + votes[i].getId());
                                 voteMap.notify();
                             }
                         }
@@ -217,11 +217,11 @@ public class FLETest extends ZKTestCase 
                          * vote if the leader takes too long to respond.
                          */
                         synchronized(FLETest.this){
-                            if (leader != votes[i].id) FLETest.this.wait(3000);
+                            if (leader != votes[i].getId()) FLETest.this.wait(3000);
 
-                            LOG.info("The leader: " + leader + " and my vote " + votes[i].id);
+                            LOG.info("The leader: " + leader + " and my vote " + votes[i].getId());
                             synchronized(voteMap){
-                                if (leader == votes[i].id) {
+                                if (leader == votes[i].getId()) {
                                     synchronized(finalObj){
                                         successCount++;
                                         if(successCount > (count/2)) finalObj.notify();
@@ -231,7 +231,7 @@ public class FLETest extends ZKTestCase 
                                     HashSet<TestVote> hs = voteMap.get(lc);
                                     TestVote toRemove = null;
                                     for(TestVote tv : hs){
-                                        if(v.id == i){
+                                        if(v.getId() == i){
                                             toRemove = tv;
                                             break;
                                         }

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java Thu Jun 16 23:34:30 2011
@@ -130,7 +130,7 @@ public class FLEZeroWeightTest extends Z
                      */
                     peer.setCurrentVote(v);
 
-                    LOG.info("Finished election: " + i + ", " + v.id);
+                    LOG.info("Finished election: " + i + ", " + v.getId());
                     votes[i] = v;
 
                     if((peer.getPeerState() == ServerState.LEADING) &&

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java Thu Jun 16 23:34:30 2011
@@ -86,17 +86,17 @@ public class LENonTerminateTest extends 
                     requestBytes.length);
             DatagramPacket responsePacket = new DatagramPacket(responseBytes,
                     responseBytes.length);
-            HashMap<InetSocketAddress, Vote> votes =
-                new HashMap<InetSocketAddress, Vote>(self.getVotingView().size());
             int xid = epochGen.nextInt();
             while (self.isRunning()) {
-                votes.clear();
+                HashMap<InetSocketAddress, Vote> votes =
+                    new HashMap<InetSocketAddress, Vote>(self.getVotingView().size());
+
                 requestBuffer.clear();
                 requestBuffer.putInt(xid);
                 requestPacket.setLength(4);
                 HashSet<Long> heardFrom = new HashSet<Long>();
                 for (QuorumServer server :
-                    (Collection<QuorumServer>)self.getVotingView().values())
+                    self.getVotingView().values())
                 {
                     LOG.info("Server address: " + server.addr);
                     try {
@@ -162,11 +162,11 @@ public class LENonTerminateTest extends 
                 // If no votes are received for live peers, reset to voting 
                 // for ourselves as otherwise we may hang on to a vote 
                 // for a dead peer                 
-                if (votes.size() == 0) {                    
+                if (result.numValidVotes == 0) {
                     self.setCurrentVote(new Vote(self.getId(),
                             self.getLastLoggedZxid()));
                 } else {
-                    if (result.winner.id >= 0) {
+                    if (result.winner.getId() >= 0) {
                         self.setCurrentVote(result.vote);
                         // To do: this doesn't use a quorum verifier
                         if (result.winningCount > (self.getVotingView().size() / 2)) {
@@ -182,7 +182,7 @@ public class LENonTerminateTest extends 
                              * error to be elected as a Leader.
                              */
                             if (self.getLearnerType() == LearnerType.OBSERVER) {
-                                if (current.id == self.getId()) {
+                                if (current.getId() == self.getId()) {
                                     // This should never happen!
                                     LOG.error("OBSERVER elected as leader!");
                                     Thread.sleep(100);
@@ -193,7 +193,7 @@ public class LENonTerminateTest extends 
                                     return current;
                                 }
                             } else {
-                                self.setPeerState((current.id == self.getId())
+                                self.setPeerState((current.getId() == self.getId())
                                         ? ServerState.LEADING: ServerState.FOLLOWING);
                                 if (self.getPeerState() == ServerState.FOLLOWING) {
                                     Thread.sleep(100);
@@ -276,7 +276,7 @@ public class LENonTerminateTest extends 
                  */
                 peer.setCurrentVote(v);
 
-                LOG.info("Finished election: " + i + ", " + v.id);                    
+                LOG.info("Finished election: " + i + ", " + v.getId());
             } catch (Exception e) {
                 e.printStackTrace();
             }
@@ -369,8 +369,8 @@ public class LENonTerminateTest extends 
             responseBuffer.getInt(); // Skip the xid
             responseBuffer.putLong(2);
             
-            responseBuffer.putLong(current.id);
-            responseBuffer.putLong(current.zxid);
+            responseBuffer.putLong(current.getId());
+            responseBuffer.putLong(current.getZxid());
             packet.setData(b);
             udpSocket.send(packet);
         }

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java?rev=1136740&r1=1136739&r2=1136740&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java Thu Jun 16 23:34:30 2011
@@ -56,7 +56,7 @@ public class LETest extends ZKTestCase {
                 while(true) {
                     v = le.lookForLeader();
                     votes[i] = v;
-                    if (v.id == i) {
+                    if (v.getId() == i) {
                         synchronized(LETest.this) {
                             if (leaderDies) {
                                 leaderDies = false;
@@ -74,7 +74,7 @@ public class LETest extends ZKTestCase {
                         if (leader == -1) {
                             LETest.this.wait();
                         }
-                        if (leader == v.id) {
+                        if (leader == v.getId()) {
                             break;
                         }
                     }
@@ -122,16 +122,16 @@ public class LETest extends ZKTestCase {
                 Assert.fail("Threads didn't join");
             }
         }
-        long id = votes[0].id;
+        long id = votes[0].getId();
         for(int i = 1; i < votes.length; i++) {
             if (votes[i] == null) {
                 Assert.fail("Thread " + i + " had a null vote");
             }
-            if (votes[i].id != id) {
-                if (allowOneBadLeader && votes[i].id == i) {
+            if (votes[i].getId() != id) {
+                if (allowOneBadLeader && votes[i].getId() == i) {
                     allowOneBadLeader = false;
                 } else {
-                    Assert.fail("Thread " + i + " got " + votes[i].id + " expected " + id);
+                    Assert.fail("Thread " + i + " got " + votes[i].getId() + " expected " + id);
                 }
             }
         }