You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ha...@apache.org on 2018/06/15 05:41:11 UTC

zookeeper git commit: ZOOKEEPER-3063: Track outstanding changes with ArrayDeque

Repository: zookeeper
Updated Branches:
  refs/heads/master a2623a625 -> d6490d590


ZOOKEEPER-3063: Track outstanding changes with ArrayDeque

Changed outstandingChanges from ArrayList to ArrayDeque.

Author: Yisong Yue <yi...@fb.com>

Reviewers: Michael Han <ha...@apache.org>

Closes #543 from yisong-yue/ZOOKEEPER-3063


Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/d6490d59
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/d6490d59
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/d6490d59

Branch: refs/heads/master
Commit: d6490d590725345638b5973a97e6e8d8bee67d22
Parents: a2623a6
Author: Yisong Yue <yi...@fb.com>
Authored: Thu Jun 14 22:41:07 2018 -0700
Committer: Michael Han <ha...@apache.org>
Committed: Thu Jun 14 22:41:07 2018 -0700

----------------------------------------------------------------------
 .../apache/zookeeper/server/FinalRequestProcessor.java    |  4 ++--
 .../org/apache/zookeeper/server/PrepRequestProcessor.java | 10 +++++-----
 .../main/org/apache/zookeeper/server/ZooKeeperServer.java |  7 +++++--
 3 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zookeeper/blob/d6490d59/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
index 4e1d762..2f60f78 100644
--- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
+++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
@@ -110,8 +110,8 @@ public class FinalRequestProcessor implements RequestProcessor {
                 Record txn = request.getTxn();
                 long zxid = hdr.getZxid();
                 while (!zks.outstandingChanges.isEmpty()
-                       && zks.outstandingChanges.get(0).zxid <= zxid) {
-                    ChangeRecord cr = zks.outstandingChanges.remove(0);
+                       && zks.outstandingChanges.peek().zxid <= zxid) {
+                    ChangeRecord cr = zks.outstandingChanges.remove();
                     if (cr.zxid < zxid) {
                         LOG.warn("Zxid outstanding " + cr.zxid
                                  + " is less than current " + zxid);

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/d6490d59/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
index f31303a..6a07320 100644
--- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
+++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
@@ -72,9 +72,9 @@ import java.io.StringReader;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.ListIterator;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
@@ -247,9 +247,9 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
     void rollbackPendingChanges(long zxid, Map<String, ChangeRecord>pendingChangeRecords) {
         synchronized (zks.outstandingChanges) {
             // Grab a list iterator starting at the END of the list so we can iterate in reverse
-            ListIterator<ChangeRecord> iter = zks.outstandingChanges.listIterator(zks.outstandingChanges.size());
-            while (iter.hasPrevious()) {
-                ChangeRecord c = iter.previous();
+            Iterator<ChangeRecord> iter = zks.outstandingChanges.descendingIterator();
+            while (iter.hasNext()) {
+                ChangeRecord c = iter.next();
                 if (c.zxid == zxid) {
                     iter.remove();
                     // Remove all outstanding changes for paths of this multi.
@@ -265,7 +265,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
                 return;
             }
 
-            long firstZxid = zks.outstandingChanges.get(0).zxid;
+            long firstZxid = zks.outstandingChanges.peek().zxid;
 
             for (ChangeRecord c : pendingChangeRecords.values()) {
                 // Don't apply any prior change records less than firstZxid.

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/d6490d59/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
index 8b9316e..ff5b3b6 100644
--- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
+++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
@@ -24,16 +24,19 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintWriter;
 import java.nio.ByteBuffer;
+
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Deque;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.Set;
 
 import javax.security.sasl.SaslException;
 
@@ -114,7 +117,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     static final private long superSecret = 0XB3415C00L;
 
     private final AtomicInteger requestsInProcess = new AtomicInteger(0);
-    final List<ChangeRecord> outstandingChanges = new ArrayList<ChangeRecord>();
+    final Deque<ChangeRecord> outstandingChanges = new ArrayDeque<>();
     // this data structure must be accessed under the outstandingChanges lock
     final Map<String, ChangeRecord> outstandingChangesForPath =
         new HashMap<String, ChangeRecord>();