瀏覽代碼

ZOOKEEPER-1265. Normalize switch cases lists on request types (Thomas Koch via phunt)

git-svn-id: https://svn.apache.org/repos/asf/zookeeper/trunk@1190431 13f79535-47bb-0310-9956-ffa450edef68
Patrick D. Hunt 13 年之前
父節點
當前提交
4ae2225de1

+ 3 - 0
CHANGES.txt

@@ -91,6 +91,9 @@ IMPROVEMENTS:
   ZOOKEEPER-1247. dead code in PrepRequestProcessor.pRequest multi case
   (Thomas Koch via phunt)
 
+  ZOOKEEPER-1265. Normalize switch cases lists on request types
+  (Thomas Koch via phunt)
+
 Release 3.4.0 - 
 
 Non-backward compatible changes:

+ 3 - 2
src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java

@@ -67,7 +67,7 @@ public class CommitProcessor extends Thread implements RequestProcessor {
     public void run() {
         ArrayList<Request> toProcess = new ArrayList<Request>();
         try {
-            Request nextPending = null;            
+            Request nextPending = null;
             while (!finished) {
                 for (Request request : toProcess) {
                     nextProcessor.processRequest(request);
@@ -118,6 +118,7 @@ public class CommitProcessor extends Thread implements RequestProcessor {
                         case OpCode.delete:
                         case OpCode.setData:
                         case OpCode.multi:
+                        case OpCode.check:
                         case OpCode.setACL:
                         case OpCode.createSession:
                         case OpCode.closeSession:
@@ -163,7 +164,7 @@ public class CommitProcessor extends Thread implements RequestProcessor {
         if (LOG.isDebugEnabled()) {
             LOG.debug("Processing request:: " + request);
         }
-        
+
         if (!finished) {
             queuedRequests.add(request);
             notifyAll();

+ 2 - 1
src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java

@@ -67,7 +67,7 @@ public class FollowerRequestProcessor extends Thread implements
                 // the request to the leader so that we are ready to receive
                 // the response
                 nextProcessor.processRequest(request);
-                
+
                 // We now ship the request to the leader. As with all
                 // other quorum operations, sync also follows this code
                 // path, but different from others, we need to keep track
@@ -85,6 +85,7 @@ public class FollowerRequestProcessor extends Thread implements
                 case OpCode.createSession:
                 case OpCode.closeSession:
                 case OpCode.multi:
+                case OpCode.check:
                     zks.getFollower().request(request);
                     break;
                 }

+ 6 - 5
src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java

@@ -40,15 +40,15 @@ public class ObserverRequestProcessor extends Thread implements
 
     RequestProcessor nextProcessor;
 
-    // We keep a queue of requests. As requests get submitted they are 
-    // stored here. The queue is drained in the run() method. 
+    // We keep a queue of requests. As requests get submitted they are
+    // stored here. The queue is drained in the run() method.
     LinkedBlockingQueue<Request> queuedRequests = new LinkedBlockingQueue<Request>();
 
     boolean finished = false;
 
     /**
      * Constructor - takes an ObserverZooKeeperServer to associate with
-     * and the next processor to pass requests to after we're finished. 
+     * and the next processor to pass requests to after we're finished.
      * @param zks
      * @param nextProcessor
      */
@@ -75,7 +75,7 @@ public class ObserverRequestProcessor extends Thread implements
                 // the request to the leader so that we are ready to receive
                 // the response
                 nextProcessor.processRequest(request);
-                
+
                 // We now ship the request to the leader. As with all
                 // other quorum operations, sync also follows this code
                 // path, but different from others, we need to keep track
@@ -93,6 +93,7 @@ public class ObserverRequestProcessor extends Thread implements
                 case OpCode.createSession:
                 case OpCode.closeSession:
                 case OpCode.multi:
+                case OpCode.check:
                     zks.getObserver().request(request);
                     break;
                 }
@@ -104,7 +105,7 @@ public class ObserverRequestProcessor extends Thread implements
     }
 
     /**
-     * Simply queue the request, which will be processed in FIFO order. 
+     * Simply queue the request, which will be processed in FIFO order.
      */
     public void processRequest(Request request) {
         if (!finished) {

+ 5 - 4
src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java

@@ -41,13 +41,13 @@ public class ReadOnlyRequestProcessor extends Thread implements RequestProcessor
 
     private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyRequestProcessor.class);
 
-    private LinkedBlockingQueue<Request> queuedRequests = new LinkedBlockingQueue<Request>();
+    private final LinkedBlockingQueue<Request> queuedRequests = new LinkedBlockingQueue<Request>();
 
     private boolean finished = false;
 
-    private RequestProcessor nextProcessor;
+    private final RequestProcessor nextProcessor;
 
-    private ZooKeeperServer zks;
+    private final ZooKeeperServer zks;
 
     public ReadOnlyRequestProcessor(ZooKeeperServer zks, RequestProcessor nextProcessor) {
         super("ReadOnlyRequestProcessor:" + zks.getServerId());
@@ -79,7 +79,8 @@ public class ReadOnlyRequestProcessor extends Thread implements RequestProcessor
                 case OpCode.delete:
                 case OpCode.setData:
                 case OpCode.setACL:
-
+                case OpCode.multi:
+                case OpCode.check:
                     ReplyHeader hdr = new ReplyHeader(request.cxid, zks.getZKDatabase()
                             .getDataTreeLastProcessedZxid(), Code.NOTREADONLY.intValue());
                     try {