|
@@ -21,11 +21,14 @@ package org.apache.zookeeper.server;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.io.PrintWriter;
|
|
|
+import java.util.ArrayDeque;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.Iterator;
|
|
|
-import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Queue;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
@@ -84,7 +87,7 @@ public class ZKDatabase {
|
|
|
|
|
|
public static final int commitLogCount = 500;
|
|
|
protected static int commitLogBuffer = 700;
|
|
|
- protected LinkedList<Proposal> committedLog = new LinkedList<Proposal>();
|
|
|
+ protected Queue<Proposal> committedLog = new ArrayDeque<>();
|
|
|
protected ReentrantReadWriteLock logLock = new ReentrantReadWriteLock();
|
|
|
volatile private boolean initialized = false;
|
|
|
|
|
@@ -187,18 +190,21 @@ public class ZKDatabase {
|
|
|
}
|
|
|
|
|
|
|
|
|
- public synchronized List<Proposal> getCommittedLog() {
|
|
|
+ public synchronized Collection<Proposal> getCommittedLog() {
|
|
|
+ final Collection<Proposal> result;
|
|
|
ReadLock rl = logLock.readLock();
|
|
|
- // only make a copy if this thread isn't already holding a lock
|
|
|
- if(logLock.getReadHoldCount() <=0) {
|
|
|
+ // make a copy if this thread is not already holding a lock
|
|
|
+ if (logLock.getReadHoldCount() > 0) {
|
|
|
+ result = this.committedLog;
|
|
|
+ } else {
|
|
|
+ rl.lock();
|
|
|
try {
|
|
|
- rl.lock();
|
|
|
- return new LinkedList<Proposal>(this.committedLog);
|
|
|
+ result = new ArrayList<>(this.committedLog);
|
|
|
} finally {
|
|
|
rl.unlock();
|
|
|
}
|
|
|
}
|
|
|
- return this.committedLog;
|
|
|
+ return Collections.unmodifiableCollection(result);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -281,8 +287,8 @@ public class ZKDatabase {
|
|
|
try {
|
|
|
wl.lock();
|
|
|
if (committedLog.size() > commitLogCount) {
|
|
|
- committedLog.removeFirst();
|
|
|
- minCommittedLog = committedLog.getFirst().packet.getZxid();
|
|
|
+ committedLog.remove();
|
|
|
+ minCommittedLog = committedLog.peek().packet.getZxid();
|
|
|
}
|
|
|
if (committedLog.isEmpty()) {
|
|
|
minCommittedLog = request.zxid;
|