Author: mahadev
Date: Wed Apr  8 21:09:42 2009
New Revision: 763395

URL: http://svn.apache.org/viewvc?rev=763395&view=rev
Log:
ZOOKEEPER-370. Fix critical problems reported by findbugs. (flavio via mahadev)

Modified:
    hadoop/zookeeper/trunk/CHANGES.txt
    
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java
    
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=763395&r1=763394&r2=763395&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Wed Apr  8 21:09:42 2009
@@ -41,6 +41,8 @@
 
   ZOOKEEPER-363. NPE when recovering ledger with no hint. (flavio via mahadev)
 
+  ZOOKEEPER-370. Fix critical problems reported by findbugs. (flavio via 
mahadev)
+
 IMPROVEMENTS:
   ZOOKEEPER-308. improve the atomic broadcast performance 3x.
   (breed via mahadev)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java?rev=763395&r1=763394&r2=763395&view=diff
==============================================================================
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java
 Wed Apr  8 21:09:42 2009
@@ -39,7 +39,8 @@
 
 public class LeaderElection implements Election  {
     private static final Logger LOG = Logger.getLogger(LeaderElection.class);
-
+    private static Random epochGen = new Random();
+    
     QuorumPeer self;
 
     public LeaderElection(QuorumPeer self) {
@@ -130,7 +131,7 @@
                 responseBytes.length);
         HashMap<InetSocketAddress, Vote> votes = new 
HashMap<InetSocketAddress, Vote>(
                 self.quorumPeers.size());
-        int xid = new Random().nextInt();
+        int xid = epochGen.nextInt();
         while (self.running) {
             votes.clear();
             requestBuffer.clear();

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java?rev=763395&r1=763394&r2=763395&view=diff
==============================================================================
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
 Wed Apr  8 21:09:42 2009
@@ -72,12 +72,7 @@
      * Packet size
      */
     int packetSize;
-
-    /*
-     * Port to listen on
-     */
-    int port;
-
+    
     /*
      * Challenge to initiate connections
      */
@@ -126,27 +121,11 @@
         this.senderWorkerMap = new ConcurrentHashMap<Long, SendWorker>();
         this.self = self;
 
-        // Generates a challenge to guarantee one connection between pairs of
-        // servers
-        //genChallenge();
-
         // Starts listener thread that waits for connection requests 
         listener = new Listener();
         listener.start();
     }
 
-    void genChallenge() {
-        try{
-            Random rand = new Random(System.currentTimeMillis()
-                + InetAddress.getLocalHost().hashCode());
-            long newValue = rand.nextLong();
-            challenge = newValue;
-        } catch(UnknownHostException e){
-            LOG.error("Cannot resolve local address");
-            challenge = 0;
-        }
-    }
-
     /**
      * If this server has initiated the connection, then it gives up on the
      * connection if it loses challenge. Otherwise, it keeps the connection.
@@ -177,30 +156,25 @@
             }
         // Otherwise proceed with the connection
         } else {
-                if (s != null) {
-                    SendWorker sw = new SendWorker(s, sid);
-                    RecvWorker rw = new RecvWorker(s, sid);
-                    sw.setRecv(rw);
-
-                    if (senderWorkerMap
-                            .containsKey(sid)) {
-                        senderWorkerMap.get(sid).finish();
-                    }
-
-                    if (!queueSendMap.containsKey(sid)) {
-                        queueSendMap.put(sid, new 
ArrayBlockingQueue<ByteBuffer>(
-                                CAPACITY));
-                    }
+               SendWorker sw = new SendWorker(s, sid);
+               RecvWorker rw = new RecvWorker(s, sid);
+               sw.setRecv(rw);
+               
+               if (senderWorkerMap
+                               .containsKey(sid)) {
+                       senderWorkerMap.get(sid).finish();
+               }
+
+               if (!queueSendMap.containsKey(sid)) {
+                       queueSendMap.put(sid, new 
ArrayBlockingQueue<ByteBuffer>(
+                                       CAPACITY));
+               }
                     
-                    senderWorkerMap.put(sid, sw);
-                    sw.start();
-                    rw.start();
+               senderWorkerMap.put(sid, sw);
+               sw.start();
+               rw.start();
 
-                    return true;
-                } else {
-                    LOG.warn("Channel null");
-                    return false;
-                }
+               return true;
             
         }
         return false;
@@ -247,30 +221,24 @@
             }
         //Otherwise start worker threads to receive data.
         } else {
-            
-                if (s != null) {
-                    SendWorker sw = new SendWorker(s, sid);
-                    RecvWorker rw = new RecvWorker(s, sid);
-                    sw.setRecv(rw);
-
-                    if (senderWorkerMap.containsKey(sid)) {
-                        senderWorkerMap.get(sid).finish();
-                    }
+               SendWorker sw = new SendWorker(s, sid);
+               RecvWorker rw = new RecvWorker(s, sid);
+               sw.setRecv(rw);
+
+               if (senderWorkerMap.containsKey(sid)) {
+                       senderWorkerMap.get(sid).finish();
+               }
                     
-                    senderWorkerMap.put(sid, sw);
+               senderWorkerMap.put(sid, sw);
                     
-                    if (!queueSendMap.containsKey(sid)) {
-                        queueSendMap.put(sid, new 
ArrayBlockingQueue<ByteBuffer>(
-                                CAPACITY));
-                    }      
-                    sw.start();
-                    rw.start();
+               if (!queueSendMap.containsKey(sid)) {
+                       queueSendMap.put(sid, new 
ArrayBlockingQueue<ByteBuffer>(
+                                       CAPACITY));
+               }      
+               sw.start();
+               rw.start();
 
-                    return true;
-                } else {
-                    LOG.warn("Channel null");
-                    return false;
-                }
+               return true;    
         }
         return false;
     }


Reply via email to