|
@@ -19,11 +19,12 @@
|
|
|
package org.apache.zookeeper.server.quorum;
|
|
|
|
|
|
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
|
|
|
-import java.io.IOException;
|
|
|
import java.util.ArrayDeque;
|
|
|
import java.util.Deque;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Map;
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.Set;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
|
|
|
@@ -82,6 +83,12 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
/** Default worker pool shutdown timeout in ms: 5000 (5s) */
|
|
|
public static final String ZOOKEEPER_COMMIT_PROC_SHUTDOWN_TIMEOUT =
|
|
|
"zookeeper.commitProcessor.shutdownTimeout";
|
|
|
+ /** Default max read batch size: -1 to disable the feature */
|
|
|
+ public static final String ZOOKEEPER_COMMIT_PROC_MAX_READ_BATCH_SIZE =
|
|
|
+ "zookeeper.commitProcessor.maxReadBatchSize";
|
|
|
+ /** Default max commit batch size: 1 */
|
|
|
+ public static final String ZOOKEEPER_COMMIT_PROC_MAX_COMMIT_BATCH_SIZE =
|
|
|
+ "zookeeper.commitProcessor.maxCommitBatchSize";
|
|
|
|
|
|
/**
|
|
|
* Incoming requests.
|
|
@@ -89,6 +96,13 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
protected LinkedBlockingQueue<Request> queuedRequests =
|
|
|
new LinkedBlockingQueue<Request>();
|
|
|
|
|
|
+ /**
|
|
|
+ * Incoming requests that are waiting on a commit,
|
|
|
+ * contained in order of arrival
|
|
|
+ */
|
|
|
+ protected final LinkedBlockingQueue<Request> queuedWriteRequests =
|
|
|
+ new LinkedBlockingQueue<>();
|
|
|
+
|
|
|
/**
|
|
|
* The number of read requests currently held in all session queues
|
|
|
*/
|
|
@@ -125,6 +139,23 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
protected WorkerService workerPool;
|
|
|
private Object emptyPoolSync = new Object();
|
|
|
|
|
|
+ /**
|
|
|
+ * Max number of reads to process from queuedRequests before switching to
|
|
|
+ * processing commits. If the value is negative, we switch whenever we have
|
|
|
+ * a local write, and pending commits.
|
|
|
+ * A high read batch size will delay commit processing causing us to
|
|
|
+ * serve stale data.
|
|
|
+ */
|
|
|
+ private static volatile int maxReadBatchSize;
|
|
|
+ /**
|
|
|
+ * Max number of commits to process before processing reads. We will try to
|
|
|
+ * process as many remote/local commits as we can till we reach this
|
|
|
+ * count.
|
|
|
+ * A high commit batch size will delay reads while processing more commits.
|
|
|
+ * A low commit batch size will favor reads.
|
|
|
+ */
|
|
|
+ private static volatile int maxCommitBatchSize;
|
|
|
+
|
|
|
/**
|
|
|
* This flag indicates whether we need to wait for a response to come back from the
|
|
|
* leader or we just let the sync operation flow through like a read. The flag will
|
|
@@ -209,22 +240,28 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
|
|
|
/*
|
|
|
* Processing up to requestsToProcess requests from the incoming
|
|
|
- * queue (queuedRequests), possibly less if a committed request
|
|
|
- * is present along with a pending local write. After the loop,
|
|
|
- * we process one committed request if commitIsWaiting.
|
|
|
+ * queue (queuedRequests). If maxReadBatchSize is set then no
|
|
|
+ * commits will be processed until maxReadBatchSize number of
|
|
|
+ * reads are processed (or no more reads remain in the queue).
|
|
|
+ * After the loop a single committed request is processed if
|
|
|
+ * one is waiting (or a batch of commits if maxCommitBatchSize
|
|
|
+ * is set).
|
|
|
*/
|
|
|
- Request request = null;
|
|
|
+ Request request;
|
|
|
+ int readsProcessed = 0;
|
|
|
while (!stopped && requestsToProcess > 0
|
|
|
+ && (maxReadBatchSize < 0 || readsProcessed <= maxReadBatchSize)
|
|
|
&& (request = queuedRequests.poll()) != null) {
|
|
|
requestsToProcess--;
|
|
|
if (needCommit(request)
|
|
|
|| pendingRequests.containsKey(request.sessionId)) {
|
|
|
// Add request to pending
|
|
|
- pendingRequests
|
|
|
- .computeIfAbsent(request.sessionId, sid -> new ArrayDeque<>())
|
|
|
- .add(request);
|
|
|
- ServerMetrics.getMetrics().REQUESTS_IN_SESSION_QUEUE.add(pendingRequests.get(request.sessionId).size());
|
|
|
+ Deque<Request> requests =
|
|
|
+ pendingRequests.computeIfAbsent(request.sessionId, sid -> new ArrayDeque<>());
|
|
|
+ requests.addLast(request);
|
|
|
+ ServerMetrics.getMetrics().REQUESTS_IN_SESSION_QUEUE.add(requests.size());
|
|
|
} else {
|
|
|
+ readsProcessed++;
|
|
|
numReadQueuedRequests.decrementAndGet();
|
|
|
sendToNextProcessor(request);
|
|
|
}
|
|
@@ -237,9 +274,10 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
* the queue, so if we have a pending request and a
|
|
|
* committed request, the committed request must be for that
|
|
|
* pending write or for a write originating at a different
|
|
|
- * server.
|
|
|
+ * server. We skip this if maxReadBatchSize is set.
|
|
|
*/
|
|
|
- if (!pendingRequests.isEmpty() && !committedRequests.isEmpty()){
|
|
|
+ if (maxReadBatchSize < 0 &&
|
|
|
+ !pendingRequests.isEmpty() && !committedRequests.isEmpty()){
|
|
|
/*
|
|
|
* We set commitIsWaiting so that we won't check
|
|
|
* committedRequests again.
|
|
@@ -248,91 +286,111 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
+ ServerMetrics.getMetrics().READS_ISSUED_IN_COMMIT_PROC.add(readsProcessed);
|
|
|
|
|
|
- // Handle a single committed request
|
|
|
- if (commitIsWaiting && !stopped){
|
|
|
+ if (!commitIsWaiting) {
|
|
|
+ commitIsWaiting = !committedRequests.isEmpty();
|
|
|
+ }
|
|
|
+
|
|
|
+ /*
|
|
|
+ * Handle commits, if any.
|
|
|
+ */
|
|
|
+ if (commitIsWaiting && !stopped) {
|
|
|
+ /*
|
|
|
+ * Drain outstanding reads
|
|
|
+ */
|
|
|
waitForEmptyPool();
|
|
|
|
|
|
- if (stopped){
|
|
|
+ if (stopped) {
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- // Process committed head
|
|
|
- if ((request = committedRequests.poll()) == null) {
|
|
|
- throw new IOException("Error: committed head is null");
|
|
|
- }
|
|
|
+ int commitsToProcess = maxCommitBatchSize;
|
|
|
|
|
|
/*
|
|
|
- * Check if request is pending, if so, update it with the committed info
|
|
|
+ * Loop through all the commits, and try to drain them.
|
|
|
*/
|
|
|
- Deque<Request> sessionQueue = pendingRequests
|
|
|
- .get(request.sessionId);
|
|
|
- ServerMetrics.getMetrics().PENDING_SESSION_QUEUE_SIZE.add(pendingRequests.size());
|
|
|
- if (sessionQueue != null) {
|
|
|
- ServerMetrics.getMetrics().REQUESTS_IN_SESSION_QUEUE.add(sessionQueue.size());
|
|
|
- // If session queue != null, then it is also not empty.
|
|
|
- Request topPending = sessionQueue.poll();
|
|
|
- if (request.cxid != topPending.cxid) {
|
|
|
- /*
|
|
|
- * TL;DR - we should not encounter this scenario often under normal load.
|
|
|
- * We pass the commit to the next processor and put the pending back with a warning.
|
|
|
- *
|
|
|
- * Generally, we can get commit requests that are not at the queue head after
|
|
|
- * a session moved (see ZOOKEEPER-2684). Let's denote the previous server of the session
|
|
|
- * with A, and the server that the session moved to with B (keep in mind that it is
|
|
|
- * possible that the session already moved from B to a new server C, and maybe C=A).
|
|
|
- * 1. If request.cxid < topPending.cxid : this means that the session requested this update
|
|
|
- * from A, then moved to B (i.e., which is us), and now B receives the commit
|
|
|
- * for the update after the session already performed several operations in B
|
|
|
- * (and therefore its cxid is higher than that old request).
|
|
|
- * 2. If request.cxid > topPending.cxid : this means that the session requested an updated
|
|
|
- * from B with cxid that is bigger than the one we know therefore in this case we
|
|
|
- * are A, and we lost the connection to the session. Given that we are waiting for a commit
|
|
|
- * for that update, it means that we already sent the request to the leader and it will
|
|
|
- * be committed at some point (in this case the order of cxid won't follow zxid, since zxid
|
|
|
- * is an increasing order). It is not safe for us to delete the session's queue at this
|
|
|
- * point, since it is possible that the session has newer requests in it after it moved
|
|
|
- * back to us. We just leave the queue as it is, and once the commit arrives (for the old
|
|
|
- * request), the finalRequestProcessor will see a closed cnxn handle, and just won't send a
|
|
|
- * response.
|
|
|
- * Also note that we don't have a local session, therefore we treat the request
|
|
|
- * like any other commit for a remote request, i.e., we perform the update without sending
|
|
|
- * a response.
|
|
|
- */
|
|
|
- LOG.warn("Got request " + request +
|
|
|
- " but we are expecting request " + topPending);
|
|
|
- sessionQueue.addFirst(topPending);
|
|
|
- } else {
|
|
|
+ Set<Long> queuesToDrain = new HashSet<>();
|
|
|
+ long startWriteTime = Time.currentElapsedTime();
|
|
|
+ int commitsProcessed = 0;
|
|
|
+ while (commitIsWaiting && !stopped && commitsToProcess > 0) {
|
|
|
+
|
|
|
+ // Process committed head
|
|
|
+ request = committedRequests.peek();
|
|
|
+
|
|
|
+ /*
|
|
|
+ * Check if this is a local write request is pending,
|
|
|
+ * if so, update it with the committed info. If the commit matches
|
|
|
+ * the first write queued in the blockedRequestQueue, we know this is
|
|
|
+ * a commit for a local write, as commits are received in order. Else
|
|
|
+ * it must be a commit for a remote write.
|
|
|
+ */
|
|
|
+ if (!queuedWriteRequests.isEmpty() &&
|
|
|
+ queuedWriteRequests.peek().sessionId == request.sessionId &&
|
|
|
+ queuedWriteRequests.peek().cxid == request.cxid) {
|
|
|
/*
|
|
|
- * Generally, we want to send to the next processor our version of the request,
|
|
|
- * since it contains the session information that is needed for post update processing.
|
|
|
- * In more details, when a request is in the local queue, there is (or could be) a client
|
|
|
- * attached to this server waiting for a response, and there is other bookkeeping of
|
|
|
- * requests that are outstanding and have originated from this server
|
|
|
- * (e.g., for setting the max outstanding requests) - we need to update this info when an
|
|
|
- * outstanding request completes. Note that in the other case (above), the operation
|
|
|
- * originated from a different server and there is no local bookkeeping or a local client
|
|
|
- * session that needs to be notified.
|
|
|
+ * Commit matches the earliest write in our write queue.
|
|
|
*/
|
|
|
- topPending.setHdr(request.getHdr());
|
|
|
- topPending.setTxn(request.getTxn());
|
|
|
- topPending.zxid = request.zxid;
|
|
|
- topPending.commitRecvTime = request.commitRecvTime;
|
|
|
- request = topPending;
|
|
|
-
|
|
|
- // Only decrement if we take a request off the queue.
|
|
|
- numWriteQueuedRequests.decrementAndGet();
|
|
|
+ Deque<Request> sessionQueue = pendingRequests
|
|
|
+ .get(request.sessionId);
|
|
|
+ ServerMetrics.getMetrics().PENDING_SESSION_QUEUE_SIZE.add(pendingRequests.size());
|
|
|
+ if (sessionQueue == null || sessionQueue.isEmpty() || !needCommit(sessionQueue.peek())) {
|
|
|
+ /*
|
|
|
+ * Can't process this write yet.
|
|
|
+ * Either there are reads pending in this session, or we
|
|
|
+ * haven't gotten to this write yet.
|
|
|
+ */
|
|
|
+ break;
|
|
|
+ } else {
|
|
|
+ ServerMetrics.getMetrics().REQUESTS_IN_SESSION_QUEUE.add(sessionQueue.size());
|
|
|
+ // If session queue != null, then it is also not empty.
|
|
|
+ Request topPending = sessionQueue.poll();
|
|
|
+ /*
|
|
|
+ * Generally, we want to send to the next processor our version of the request,
|
|
|
+ * since it contains the session information that is needed for post update processing.
|
|
|
+ * In more details, when a request is in the local queue, there is (or could be) a client
|
|
|
+ * attached to this server waiting for a response, and there is other bookkeeping of
|
|
|
+ * requests that are outstanding and have originated from this server
|
|
|
+ * (e.g., for setting the max outstanding requests) - we need to update this info when an
|
|
|
+ * outstanding request completes. Note that in the other case, the operation
|
|
|
+ * originated from a different server and there is no local bookkeeping or a local client
|
|
|
+ * session that needs to be notified.
|
|
|
+ */
|
|
|
+ topPending.setHdr(request.getHdr());
|
|
|
+ topPending.setTxn(request.getTxn());
|
|
|
+ topPending.zxid = request.zxid;
|
|
|
+ topPending.commitRecvTime = request.commitRecvTime;
|
|
|
+ request = topPending;
|
|
|
+ // Only decrement if we take a request off the queue.
|
|
|
+ numWriteQueuedRequests.decrementAndGet();
|
|
|
+ queuedWriteRequests.poll();
|
|
|
+ queuesToDrain.add(request.sessionId);
|
|
|
+ }
|
|
|
}
|
|
|
- }
|
|
|
+ /*
|
|
|
+ * Pull the request off the commit queue, now that we are going
|
|
|
+ * to process it.
|
|
|
+ */
|
|
|
+ committedRequests.remove();
|
|
|
+ commitsToProcess--;
|
|
|
+ commitsProcessed++;
|
|
|
|
|
|
- sendToNextProcessor(request);
|
|
|
- waitForEmptyPool();
|
|
|
+ // Process the write inline.
|
|
|
+ processWrite(request);
|
|
|
+
|
|
|
+ commitIsWaiting = !committedRequests.isEmpty();
|
|
|
+ }
|
|
|
+ ServerMetrics.getMetrics().WRITE_BATCH_TIME_IN_COMMIT_PROCESSOR.add(
|
|
|
+ Time.currentElapsedTime() - startWriteTime);
|
|
|
+ ServerMetrics.getMetrics().WRITES_ISSUED_IN_COMMIT_PROC.add(commitsProcessed);
|
|
|
|
|
|
/*
|
|
|
- * Process following reads if any, remove session queue if
|
|
|
+ * Process following reads if any, remove session queue(s) if
|
|
|
* empty.
|
|
|
*/
|
|
|
- if (sessionQueue != null) {
|
|
|
+ readsProcessed = 0;
|
|
|
+ for (Long sessionId : queuesToDrain) {
|
|
|
+ Deque<Request> sessionQueue = pendingRequests.get(sessionId);
|
|
|
int readsAfterWrite = 0;
|
|
|
while (!stopped && !sessionQueue.isEmpty()
|
|
|
&& !needCommit(sessionQueue.peek())) {
|
|
@@ -341,12 +399,15 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
readsAfterWrite++;
|
|
|
}
|
|
|
ServerMetrics.getMetrics().READS_AFTER_WRITE_IN_SESSION_QUEUE.add(readsAfterWrite);
|
|
|
+ readsProcessed += readsAfterWrite;
|
|
|
|
|
|
// Remove empty queues
|
|
|
if (sessionQueue.isEmpty()) {
|
|
|
- pendingRequests.remove(request.sessionId);
|
|
|
+ pendingRequests.remove(sessionId);
|
|
|
}
|
|
|
}
|
|
|
+ ServerMetrics.getMetrics().SESSION_QUEUES_DRAINED.add(queuesToDrain.size());
|
|
|
+ ServerMetrics.getMetrics().READ_ISSUED_FROM_SESSION_QUEUE.add(readsProcessed);
|
|
|
}
|
|
|
|
|
|
ServerMetrics.getMetrics().COMMIT_PROCESS_TIME.add(Time.currentElapsedTime() - time);
|
|
@@ -388,6 +449,8 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
workerShutdownTimeoutMS = Long.getLong(
|
|
|
ZOOKEEPER_COMMIT_PROC_SHUTDOWN_TIMEOUT, 5000);
|
|
|
|
|
|
+ initBatchSizes();
|
|
|
+
|
|
|
LOG.info("Configuring CommitProcessor with "
|
|
|
+ (numWorkerThreads > 0 ? numWorkerThreads : "no")
|
|
|
+ " worker threads.");
|
|
@@ -409,6 +472,78 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
workerPool.schedule(new CommitWorkRequest(request), request.sessionId);
|
|
|
}
|
|
|
|
|
|
+ private void processWrite(Request request) throws RequestProcessorException {
|
|
|
+ processCommitMetrics(request, true);
|
|
|
+
|
|
|
+ long timeBeforeFinalProc = Time.currentElapsedTime();
|
|
|
+ nextProcessor.processRequest(request);
|
|
|
+ ServerMetrics.getMetrics().WRITE_FINAL_PROC_TIME.add(
|
|
|
+ Time.currentElapsedTime() - timeBeforeFinalProc);
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void initBatchSizes() {
|
|
|
+ maxReadBatchSize = Integer.getInteger(
|
|
|
+ ZOOKEEPER_COMMIT_PROC_MAX_READ_BATCH_SIZE, -1);
|
|
|
+ maxCommitBatchSize = Integer.getInteger(
|
|
|
+ ZOOKEEPER_COMMIT_PROC_MAX_COMMIT_BATCH_SIZE, 1);
|
|
|
+
|
|
|
+ if (maxCommitBatchSize <= 0) {
|
|
|
+ String errorMsg = "maxCommitBatchSize must be positive, was " +
|
|
|
+ maxCommitBatchSize;
|
|
|
+ throw new IllegalArgumentException(errorMsg);
|
|
|
+ }
|
|
|
+
|
|
|
+ LOG.info("Configuring CommitProcessor with readBatchSize {} commitBatchSize {}",
|
|
|
+ maxReadBatchSize,
|
|
|
+ maxCommitBatchSize);
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void processCommitMetrics(Request request, boolean isWrite) {
|
|
|
+ if (isWrite) {
|
|
|
+ if (request.commitProcQueueStartTime != -1 &&
|
|
|
+ request.commitRecvTime != -1) {
|
|
|
+ // Locally issued writes.
|
|
|
+ long currentTime = Time.currentElapsedTime();
|
|
|
+ ServerMetrics.getMetrics().WRITE_COMMITPROC_TIME.add(currentTime -
|
|
|
+ request.commitProcQueueStartTime);
|
|
|
+ ServerMetrics.getMetrics().LOCAL_WRITE_COMMITTED_TIME.add(currentTime -
|
|
|
+ request.commitRecvTime);
|
|
|
+ } else if (request.commitRecvTime != -1) {
|
|
|
+ // Writes issued by other servers.
|
|
|
+ ServerMetrics.getMetrics().SERVER_WRITE_COMMITTED_TIME.add(
|
|
|
+ Time.currentElapsedTime() - request.commitRecvTime);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ if (request.commitProcQueueStartTime != -1) {
|
|
|
+ ServerMetrics.getMetrics().READ_COMMITPROC_TIME.add(
|
|
|
+ Time.currentElapsedTime() -
|
|
|
+ request.commitProcQueueStartTime);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public static int getMaxReadBatchSize() {
|
|
|
+ return maxReadBatchSize;
|
|
|
+ }
|
|
|
+
|
|
|
+ public static int getMaxCommitBatchSize() {
|
|
|
+ return maxCommitBatchSize;
|
|
|
+ }
|
|
|
+
|
|
|
+ public static void setMaxReadBatchSize(int size) {
|
|
|
+ maxReadBatchSize = size;
|
|
|
+ LOG.info("Configuring CommitProcessor with readBatchSize {}",
|
|
|
+ maxReadBatchSize);
|
|
|
+ }
|
|
|
+
|
|
|
+ public static void setMaxCommitBatchSize(int size) {
|
|
|
+ if (size > 0) {
|
|
|
+ maxCommitBatchSize = size;
|
|
|
+ LOG.info("Configuring CommitProcessor with commitBatchSize {}",
|
|
|
+ maxCommitBatchSize);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* CommitWorkRequest is a small wrapper class to allow
|
|
|
* downstream processing to be run using the WorkerService
|
|
@@ -431,27 +566,7 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
|
|
|
public void doWork() throws RequestProcessorException {
|
|
|
try {
|
|
|
- if (needCommit(request)) {
|
|
|
- if (request.commitProcQueueStartTime != -1 &&
|
|
|
- request.commitRecvTime != -1) {
|
|
|
- // Locally issued writes.
|
|
|
- long currentTime = Time.currentElapsedTime();
|
|
|
- ServerMetrics.getMetrics().WRITE_COMMITPROC_TIME.add(currentTime -
|
|
|
- request.commitProcQueueStartTime);
|
|
|
- ServerMetrics.getMetrics().LOCAL_WRITE_COMMITTED_TIME.add(currentTime -
|
|
|
- request.commitRecvTime);
|
|
|
- } else if (request.commitRecvTime != -1) {
|
|
|
- // Writes issued by other servers.
|
|
|
- ServerMetrics.getMetrics().SERVER_WRITE_COMMITTED_TIME.add(
|
|
|
- Time.currentElapsedTime() - request.commitRecvTime);
|
|
|
- }
|
|
|
- } else {
|
|
|
- if (request.commitProcQueueStartTime != -1) {
|
|
|
- ServerMetrics.getMetrics().READ_COMMITPROC_TIME.add(
|
|
|
- Time.currentElapsedTime() -
|
|
|
- request.commitProcQueueStartTime);
|
|
|
- }
|
|
|
- }
|
|
|
+ processCommitMetrics(request, needCommit(request));
|
|
|
|
|
|
long timeBeforeFinalProc = Time.currentElapsedTime();
|
|
|
nextProcessor.processRequest(request);
|
|
@@ -508,6 +623,7 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements
|
|
|
queuedRequests.add(request);
|
|
|
// If the request will block, add it to the queue of blocking requests
|
|
|
if (needCommit(request)) {
|
|
|
+ queuedWriteRequests.add(request);
|
|
|
numWriteQueuedRequests.incrementAndGet();
|
|
|
} else {
|
|
|
numReadQueuedRequests.incrementAndGet();
|