|
@@ -20,9 +20,11 @@ package org.apache.zookeeper.server.quorum;
|
|
|
|
|
|
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
|
|
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.util.ArrayDeque;
|
|
|
|
+import java.util.Deque;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
-import java.util.LinkedList;
|
|
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
+import java.util.Queue;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
|
|
|
|
@@ -96,8 +98,8 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
* Requests that we are holding until commit comes in. Keys represent
|
|
* Requests that we are holding until commit comes in. Keys represent
|
|
* session ids, each value is a linked list of the session's requests.
|
|
* session ids, each value is a linked list of the session's requests.
|
|
*/
|
|
*/
|
|
- protected final Map<Long, LinkedList<Request>> pendingRequests =
|
|
|
|
- new HashMap<Long, LinkedList<Request>>(10000);
|
|
|
|
|
|
+ protected final Map<Long, Deque<Request>> pendingRequests =
|
|
|
|
+ new HashMap<>(10000);
|
|
|
|
|
|
/** The number of requests currently being processed */
|
|
/** The number of requests currently being processed */
|
|
protected final AtomicInteger numRequestsProcessing = new AtomicInteger(0);
|
|
protected final AtomicInteger numRequestsProcessing = new AtomicInteger(0);
|
|
@@ -200,13 +202,9 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
if (needCommit(request)
|
|
if (needCommit(request)
|
|
|| pendingRequests.containsKey(request.sessionId)) {
|
|
|| pendingRequests.containsKey(request.sessionId)) {
|
|
// Add request to pending
|
|
// Add request to pending
|
|
- LinkedList<Request> requests = pendingRequests
|
|
|
|
- .get(request.sessionId);
|
|
|
|
- if (requests == null) {
|
|
|
|
- requests = new LinkedList<Request>();
|
|
|
|
- pendingRequests.put(request.sessionId, requests);
|
|
|
|
- }
|
|
|
|
- requests.addLast(request);
|
|
|
|
|
|
+ pendingRequests
|
|
|
|
+ .computeIfAbsent(request.sessionId, sid -> new ArrayDeque<>())
|
|
|
|
+ .add(request);
|
|
}
|
|
}
|
|
else {
|
|
else {
|
|
sendToNextProcessor(request);
|
|
sendToNextProcessor(request);
|
|
@@ -248,7 +246,7 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
/*
|
|
/*
|
|
* Check if request is pending, if so, update it with the committed info
|
|
* Check if request is pending, if so, update it with the committed info
|
|
*/
|
|
*/
|
|
- LinkedList<Request> sessionQueue = pendingRequests
|
|
|
|
|
|
+ Deque<Request> sessionQueue = pendingRequests
|
|
.get(request.sessionId);
|
|
.get(request.sessionId);
|
|
if (sessionQueue != null) {
|
|
if (sessionQueue != null) {
|
|
// If session queue != null, then it is also not empty.
|
|
// If session queue != null, then it is also not empty.
|