Author: henry
Date: Fri Dec 11 22:42:19 2009
New Revision: 889843
URL: http://svn.apache.org/viewvc?rev=889843&view=rev
Log:
Committing 610
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathTrie.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Follower.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.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=889843&r1=889842&r2=889843&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Dec 11 22:42:19 2009
@@ -7,6 +7,9 @@
Backward compatible changes:
BUGFIXES:
+ ZOOKEEPER-610. cleanup final fields, esp those used for locking
+ (phunt via henry)
+
ZOOKEEPER-469. make sure CPPUNIT_CFLAGS isn't overwritten
(chris via mahadev)
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=889843&r1=889842&r2=889843&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java Fri Dec 11 22:42:19
2009
@@ -98,7 +98,8 @@
packetLen = Integer.getInteger("jute.maxbuffer", 4096 * 1024);
}
- private ArrayList<InetSocketAddress> serverAddrs = new ArrayList<InetSocketAddress>();
+ private final ArrayList<InetSocketAddress> serverAddrs =
+ new ArrayList<InetSocketAddress>();
static class AuthData {
AuthData(String scheme, byte data[]) {
@@ -111,17 +112,17 @@
byte data[];
}
- private ArrayList<AuthData> authInfo = new ArrayList<AuthData>();
+ private final ArrayList<AuthData> authInfo = new ArrayList<AuthData>();
/**
* These are the packets that have been sent and are waiting for a response.
*/
- private LinkedList<Packet> pendingQueue = new LinkedList<Packet>();
+ private final LinkedList<Packet> pendingQueue = new LinkedList<Packet>();
/**
* These are the packets that need to be sent.
*/
- private LinkedList<Packet> outgoingQueue = new LinkedList<Packet>();
+ private final LinkedList<Packet> outgoingQueue = new LinkedList<Packet>();
private int nextAddrToTry = 0;
@@ -599,7 +600,7 @@
class SendThread extends Thread {
SelectionKey sockKey;
- ByteBuffer lenBuffer = ByteBuffer.allocateDirect(4);
+ final ByteBuffer lenBuffer = ByteBuffer.allocateDirect(4);
ByteBuffer incomingBuffer = lenBuffer;
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathTrie.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathTrie.java?rev=889843&r1=889842&r2=889843&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathTrie.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathTrie.java Fri Dec
11 22:42:19 2009
@@ -48,11 +48,11 @@
/**
* the root node of PathTrie
*/
- private TrieNode rootNode ;
+ private final TrieNode rootNode ;
static class TrieNode {
boolean property = false;
- HashMap<String, TrieNode> children;
+ final HashMap<String, TrieNode> children;
TrieNode parent = null;
/**
* create a trienode with parent
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java?rev=889843&r1=889842&r2=889843&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java Fri Dec
11 22:42:19 2009
@@ -250,10 +250,11 @@
public long approximateDataSize() {
long result = 0;
for (Map.Entry<String, DataNode> entry : nodes.entrySet()) {
- synchronized (entry.getValue()) {
+ DataNode value = entry.getValue();
+ synchronized (value) {
result += entry.getKey().length();
- result += (entry.getValue().data == null ? 0
- : entry.getValue().data.length);
+ result += (value.data == null ? 0
+ : value.data.length);
}
}
return result;
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java?rev=889843&r1=889842&r2=889843&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java Fri
Dec 11 22:42:19 2009
@@ -92,19 +92,20 @@
ZooKeeperServer zks;
- ServerSocketChannel ss;
+ final ServerSocketChannel ss;
- Selector selector = Selector.open();
+ final Selector selector = Selector.open();
/**
* We use this buffer to do efficient socket I/O. Since there is a single
* sender thread per NIOServerCnxn instance, we can use a member variable to
* only allocate it once.
*/
- ByteBuffer directBuffer = ByteBuffer.allocateDirect(64 * 1024);
+ final ByteBuffer directBuffer = ByteBuffer.allocateDirect(64 * 1024);
- HashSet<NIOServerCnxn> cnxns = new HashSet<NIOServerCnxn>();
- HashMap<InetAddress, Set<NIOServerCnxn>> ipMap = new HashMap<InetAddress,
Set<NIOServerCnxn>>( );
+ final HashSet<NIOServerCnxn> cnxns = new HashSet<NIOServerCnxn>();
+ final HashMap<InetAddress, Set<NIOServerCnxn>> ipMap =
+ new HashMap<InetAddress, Set<NIOServerCnxn>>( );
int outstandingLimit = 1;
@@ -332,7 +333,7 @@
*/
static final ByteBuffer closeConn = ByteBuffer.allocate(0);
- Factory factory;
+ final Factory factory;
private final ZooKeeperServer zk;
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java?rev=889843&r1=889842&r2=889843&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
Fri Dec 11 22:42:19 2009
@@ -99,7 +99,7 @@
public static final int commitLogCount = 500;
public int commitLogBuffer = 700;
- public LinkedList<Proposal> committedLog = new LinkedList<Proposal>();
+ public final LinkedList<Proposal> committedLog = new LinkedList<Proposal>();
public long minCommittedLog, maxCommittedLog;
private DataTreeBuilder treeBuilder;
public DataTree dataTree;
@@ -107,7 +107,7 @@
private FileTxnSnapLog txnLogFactory = null;
protected ConcurrentHashMap<Long, Integer> sessionsWithTimeouts;
protected long hzxid = 0;
- final public static Exception ok = new Exception("No prob");
+ public final static Exception ok = new Exception("No prob");
protected RequestProcessor firstProcessor;
protected volatile boolean running;
@@ -118,9 +118,10 @@
static final private long superSecret = 0XB3415C00L;
int requestsInProcess;
- List<ChangeRecord> outstandingChanges = new ArrayList<ChangeRecord>();
+ final List<ChangeRecord> outstandingChanges = new ArrayList<ChangeRecord>();
// this data structure must be accessed under the outstandingChanges lock
- HashMap<String, ChangeRecord> outstandingChangesForPath = new HashMap<String,
ChangeRecord>();
+ final HashMap<String, ChangeRecord> outstandingChangesForPath =
+ new HashMap<String, ChangeRecord>();
private NIOServerCnxn.Factory serverCnxnFactory;
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java?rev=889843&r1=889842&r2=889843&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
Fri Dec 11 22:42:19 2009
@@ -189,15 +189,15 @@
private class Messenger {
- DatagramSocket mySocket;
+ final DatagramSocket mySocket;
long lastProposedLeader;
long lastProposedZxid;
long lastEpoch;
- LinkedBlockingQueue<Long> acksqueue;
- HashMap<Long, Long> challengeMap;
- HashMap<Long, Semaphore> challengeMutex;
- HashMap<Long, Semaphore> ackMutex;
- HashMap<InetSocketAddress, HashMap<Long, Long>> addrChallengeMap;
+ final LinkedBlockingQueue<Long> acksqueue;
+ final HashMap<Long, Long> challengeMap;
+ final HashMap<Long, Semaphore> challengeMutex;
+ final HashMap<Long, Semaphore> ackMutex;
+ final HashMap<InetSocketAddress, HashMap<Long, Long>> addrChallengeMap;
class WorkerReceiver implements Runnable {
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Follower.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Follower.java?rev=889843&r1=889842&r2=889843&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Follower.java
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Follower.java
Fri Dec 11 22:42:19 2009
@@ -34,7 +34,7 @@
private long lastQueued;
// This is the same object as this.zk, but we cache the downcast op
- FollowerZooKeeperServer fzk = null;
+ final FollowerZooKeeperServer fzk;
Follower(QuorumPeer self,FollowerZooKeeperServer zk) {
this.self = self;
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java?rev=889843&r1=889842&r2=889843&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java Fri
Dec 11 22:42:19 2009
@@ -68,26 +68,30 @@
}
}
- LeaderZooKeeperServer zk;
+ final LeaderZooKeeperServer zk;
- QuorumPeer self;
+ final QuorumPeer self;
// the follower acceptor thread
LearnerCnxAcceptor cnxAcceptor;
// list of all the followers
- public HashSet<LearnerHandler> learners = new HashSet<LearnerHandler>();
+ public final HashSet<LearnerHandler> learners =
+ new HashSet<LearnerHandler>();
// list of followers that are ready to follow (i.e synced with the leader)
- public HashSet<LearnerHandler> forwardingFollowers = new HashSet<LearnerHandler>();
+ public final HashSet<LearnerHandler> forwardingFollowers =
+ new HashSet<LearnerHandler>();
- protected HashSet<LearnerHandler> observingLearners = new HashSet<LearnerHandler>();
+ protected final HashSet<LearnerHandler> observingLearners =
+ new HashSet<LearnerHandler>();
//Pending sync requests
- public HashMap<Long,List<LearnerSyncRequest>> pendingSyncs = new HashMap<Long,List<LearnerSyncRequest>>();
+ public final HashMap<Long,List<LearnerSyncRequest>> pendingSyncs =
+ new HashMap<Long,List<LearnerSyncRequest>>();
//Follower counter
- AtomicLong followerCounter = new AtomicLong(-1);
+ final AtomicLong followerCounter = new AtomicLong(-1);
/**
* Adds peer to the leader.
*
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=889843&r1=889842&r2=889843&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
Fri Dec 11 22:42:19 2009
@@ -70,19 +70,19 @@
/*
* Local IP address
*/
- QuorumPeer self;
+ final QuorumPeer self;
/*
* Mapping from Peer to Thread number
*/
- ConcurrentHashMap<Long, SendWorker> senderWorkerMap;
- ConcurrentHashMap<Long, ArrayBlockingQueue<ByteBuffer>> queueSendMap;
- ConcurrentHashMap<Long, ByteBuffer> lastMessageSent;
+ final ConcurrentHashMap<Long, SendWorker> senderWorkerMap;
+ final ConcurrentHashMap<Long, ArrayBlockingQueue<ByteBuffer>> queueSendMap;
+ final ConcurrentHashMap<Long, ByteBuffer> lastMessageSent;
/*
* Reception queue
*/
- public ArrayBlockingQueue<Message> recvQueue;
+ public final ArrayBlockingQueue<Message> recvQueue;
/*
* Shutdown flag
@@ -93,7 +93,7 @@
/*
* Listener thread
*/
- public Listener listener;
+ public final Listener listener;
static public class Message {
Message(ByteBuffer buffer, long sid) {
|