Pārlūkot izejas kodu

ZOOKEEPER-1124. Multiop submitted to non-leader always fails due to timeout

git-svn-id: https://svn.apache.org/repos/asf/zookeeper/trunk@1146961 13f79535-47bb-0310-9956-ffa450edef68
Benjamin Reed 14 gadi atpakaļ
vecāks
revīzija
839ecef3ba

+ 2 - 0
CHANGES.txt

@@ -249,6 +249,8 @@ BUGFIXES:
 
   ZOOKEEPER-1063. Dubious synchronization in Zookeeper and ClientCnxnSocketNIO classes (Yanick Dufresne via breed)
 
+  ZOOKEEPER-1124. Multiop submitted to non-leader always fails due to timeout (Marshall McMullen via breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-724. Improve junit test integration - log harness information 
   (phunt via mahadev)

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

@@ -84,6 +84,7 @@ public class FollowerRequestProcessor extends Thread implements
                 case OpCode.setACL:
                 case OpCode.createSession:
                 case OpCode.closeSession:
+                case OpCode.multi:
                     zks.getFollower().request(request);
                     break;
                 }

+ 1 - 0
src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java

@@ -92,6 +92,7 @@ public class ObserverRequestProcessor extends Thread implements
                 case OpCode.setACL:
                 case OpCode.createSession:
                 case OpCode.closeSession:
+                case OpCode.multi:
                     zks.getObserver().request(request);
                     break;
                 }

+ 43 - 0
src/java/test/org/apache/zookeeper/test/QuorumTest.java

@@ -18,15 +18,20 @@
 
 package org.apache.zookeeper.test;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
@@ -424,4 +429,42 @@ public class QuorumTest extends QuorumBase {
     }
 
     // skip superhammer and clientcleanup as they are too expensive for quorum
+    
+    /**
+     * Tests if a multiop submitted to a non-leader propagates to the leader properly
+     * (see ZOOKEEPER-1124).
+     * 
+     * The test works as follows. It has a client connect to a follower and submit a multiop
+     * to the follower. It then verifies that the multiop successfully gets committed by the leader.
+     *
+     * Without the fix in ZOOKEEPER-1124, this fails with a ConnectionLoss KeeperException.
+     */
+    @Test
+    public void testMultiToFollower() throws Exception {
+        QuorumUtil qu = new QuorumUtil(1);
+        CountdownWatcher watcher = new CountdownWatcher();
+        qu.startQuorum();
+        
+        int index = 1;
+        while(qu.getPeer(index).peer.leader == null)
+            index++;
+        
+        ZooKeeper zk = new ZooKeeper(
+                "127.0.0.1:" + qu.getPeer((index == 1)?2:1).peer.getClientPort(),
+                ClientBase.CONNECTION_TIMEOUT, watcher);
+        watcher.waitForConnected(CONNECTION_TIMEOUT);
+        
+        List<OpResult> results = new ArrayList<OpResult>();
+
+        results = zk.multi(Arrays.asList(
+                Op.create("/multi0", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
+                Op.create("/multi1", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
+                Op.create("/multi2", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
+                ));
+        zk.getData("/multi0", false, null);
+        zk.getData("/multi1", false, null);
+        zk.getData("/multi2", false, null);
+
+        zk.close();
+    }
 }