You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ca...@apache.org on 2013/10/10 20:06:17 UTC

svn commit: r1531061 - in /zookeeper/trunk: CHANGES.txt src/c/tests/TestMulti.cc src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java src/java/test/org/apache/zookeeper/test/MultiAsyncTransactionTest.java

Author: camille
Date: Thu Oct 10 18:06:17 2013
New Revision: 1531061

URL: http://svn.apache.org/r1531061
Log:
ZOOKEEPER-1624. PrepRequestProcessor abort multi-operation incorrectly. (thawan via camille)

Added:
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiAsyncTransactionTest.java   (with props)
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/c/tests/TestMulti.cc
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1531061&r1=1531060&r2=1531061&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Thu Oct 10 18:06:17 2013
@@ -439,6 +439,8 @@ BUGFIXES:
 
   ZOOKEEPER-877. zkpython does not work with python3.1
   (Daniel Enman via phunt)
+  
+  ZOOKEEPER-1624. PrepRequestProcessor abort multi-operation incorrectly. (thawan via camille)
 
 IMPROVEMENTS:
 

Modified: zookeeper/trunk/src/c/tests/TestMulti.cc
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/tests/TestMulti.cc?rev=1531061&r1=1531060&r2=1531061&view=diff
==============================================================================
--- zookeeper/trunk/src/c/tests/TestMulti.cc (original)
+++ zookeeper/trunk/src/c/tests/TestMulti.cc Thu Oct 10 18:06:17 2013
@@ -177,6 +177,7 @@ class Zookeeper_multi : public CPPUNIT_N
     CPPUNIT_TEST(testMultiFail);
     CPPUNIT_TEST(testCheck);
     CPPUNIT_TEST(testWatch);
+    CPPUNIT_TEST(testSequentialNodeCreateInAsyncMulti);
 #endif
     CPPUNIT_TEST_SUITE_END();
 
@@ -244,6 +245,10 @@ public:
         count++;
     }
 
+    static void multi_completion_fn_no_assert(int rc, const void *data) {
+        count++;
+    }
+
     static void waitForMultiCompletion(int seconds) {
         time_t expires = time(0) + seconds;
         while(count == 0 && time(0) < expires) {
@@ -252,6 +257,10 @@ public:
         count--;
     }
 
+    static void resetCounter() {
+        count = 0;
+    }
+
     /**
      * Test basic multi-op create functionality 
      */
@@ -646,6 +655,47 @@ public:
         // wait for multi completion in doMultiInWatch
         waitForMultiCompletion(5);
      }
+
+     /**
+      * ZOOKEEPER-1624: PendingChanges of create sequential node request didn't
+      * get rollbacked correctly when multi-op failed. This caused
+      * create sequential node request in subsequent multi-op to failed because
+      * sequential node name generation is incorrect.
+      *
+      * The check is to make sure that each request in multi-op failed with
+      * the correct reason.
+      */
+     void testSequentialNodeCreateInAsyncMulti() {
+         int rc;
+         watchctx_t ctx;
+         zhandle_t *zk = createClient(&ctx);
+
+         int iteration = 4;
+         int nops = 2;
+
+         zoo_op_result_t results[iteration][nops];
+         zoo_op_t ops[nops];
+         zoo_create_op_init(&ops[0], "/node-",   "", 0, &ZOO_OPEN_ACL_UNSAFE, ZOO_SEQUENCE, NULL, 0);
+         zoo_create_op_init(&ops[1], "/dup", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, NULL, 0);
+         for (int i = 0; i < iteration ; ++i) {
+           rc = zoo_amulti(zk, nops, ops, results[i], multi_completion_fn_no_assert, 0);
+           CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+         }
+
+         waitForMultiCompletion(10);
+
+         CPPUNIT_ASSERT_EQUAL((int)ZOK, results[0][0].err);
+         CPPUNIT_ASSERT_EQUAL((int)ZOK, results[1][0].err);
+         CPPUNIT_ASSERT_EQUAL((int)ZOK, results[2][0].err);
+         CPPUNIT_ASSERT_EQUAL((int)ZOK, results[3][0].err);
+
+         CPPUNIT_ASSERT_EQUAL((int)ZOK, results[0][1].err);
+         CPPUNIT_ASSERT_EQUAL((int)ZNODEEXISTS, results[1][1].err);
+         CPPUNIT_ASSERT_EQUAL((int)ZNODEEXISTS, results[2][1].err);
+         CPPUNIT_ASSERT_EQUAL((int)ZNODEEXISTS, results[3][1].err);
+
+         resetCounter();
+     }
 };
 
 volatile int Zookeeper_multi::count;

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java?rev=1531061&r1=1531060&r2=1531061&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java Thu Oct 10 18:06:17 2013
@@ -215,6 +215,25 @@ public class PrepRequestProcessor extend
                 if (cr != null) {
                     pendingChangeRecords.put(path, cr);
                 }
+
+                /*
+                 * ZOOKEEPER-1624 - We need to store for parent's ChangeRecord
+                 * of the parent node of a request. So that if this is a
+                 * sequential node creation request, rollbackPendingChanges()
+                 * can restore previous parent's ChangeRecord correctly.
+                 *
+                 * Otherwise, sequential node name generation will be incorrect
+                 * for a subsequent request.
+                 */
+                int lastSlash = path.lastIndexOf('/');
+                if (lastSlash == -1 || path.indexOf('\0') != -1) {
+                    continue;
+                }
+                String parentPath = path.substring(0, lastSlash);
+                ChangeRecord parentCr = getRecordForPath(parentPath);
+                if (parentCr != null) {
+                    pendingChangeRecords.put(parentPath, parentCr);
+                }
             } catch (KeeperException.NoNodeException e) {
                 // ignore this one
             }

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiAsyncTransactionTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiAsyncTransactionTest.java?rev=1531061&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiAsyncTransactionTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiAsyncTransactionTest.java Thu Oct 10 18:06:17 2013
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.zookeeper.AsyncCallback.MultiCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+import org.apache.zookeeper.OpResult.CreateResult;
+import org.apache.zookeeper.OpResult.ErrorResult;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.Before;
+import org.junit.Test;
+
+public class MultiAsyncTransactionTest extends ClientBase {
+    private ZooKeeper zk;
+    private final AtomicInteger pendingOps = new AtomicInteger(0);
+
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        zk = createClient();
+        pendingOps.set(0);
+    }
+
+    private static class MultiResult {
+        int rc;
+        List<OpResult> results;
+    }
+
+    private void finishPendingOps() {
+        if (pendingOps.decrementAndGet() == 0) {
+            synchronized (pendingOps) {
+                pendingOps.notifyAll();
+            }
+        }
+    }
+
+    private void waitForPendingOps(int timeout) throws Exception {
+        synchronized(pendingOps) {
+            while(pendingOps.get() > 0) {
+                pendingOps.wait(timeout);
+            }
+        }
+    }
+
+    /**
+     * ZOOKEEPER-1624: PendingChanges of create sequential node request didn't
+     * get rollbacked correctly when multi-op failed. This cause
+     * create sequential node request in subsequent multi-op to failed because
+     * sequential node name generation is incorrect.
+     *
+     * The check is to make sure that each request in multi-op failed with
+     * the correct reason.
+     */
+    @Test
+    public void testSequentialNodeCreateInAsyncMulti() throws Exception {
+        final int iteration = 4;
+        final List<MultiResult> results = new ArrayList<MultiResult>();
+
+        pendingOps.set(iteration);
+
+        List<Op> ops = Arrays.asList(
+                Op.create("/node-", new byte[0], Ids.OPEN_ACL_UNSAFE,
+                          CreateMode.PERSISTENT_SEQUENTIAL),
+                Op.create("/dup", new byte[0], Ids.OPEN_ACL_UNSAFE,
+                          CreateMode.PERSISTENT));
+
+
+        for (int i = 0; i < iteration; ++i) {
+            zk.multi(ops, new MultiCallback() {
+                @Override
+                public void processResult(int rc, String path, Object ctx,
+                        List<OpResult> opResults) {
+                    MultiResult result = new MultiResult();
+                    result.results = opResults;
+                    result.rc = rc;
+                    results.add(result);
+                    finishPendingOps();
+                }
+            }, null);
+        }
+
+        waitForPendingOps(CONNECTION_TIMEOUT);
+
+        // Check that return code of all request are correct
+        assertEquals(KeeperException.Code.OK.intValue(), results.get(0).rc);
+        assertEquals(KeeperException.Code.NODEEXISTS.intValue(), results.get(1).rc);
+        assertEquals(KeeperException.Code.NODEEXISTS.intValue(), results.get(2).rc);
+        assertEquals(KeeperException.Code.NODEEXISTS.intValue(), results.get(3).rc);
+
+        // Check that the first operation is successful in all request
+        assertTrue(results.get(0).results.get(0) instanceof CreateResult);
+        assertEquals(KeeperException.Code.OK.intValue(),
+                ((ErrorResult) results.get(1).results.get(0)).getErr());
+        assertEquals(KeeperException.Code.OK.intValue(),
+                ((ErrorResult) results.get(2).results.get(0)).getErr());
+        assertEquals(KeeperException.Code.OK.intValue(),
+                ((ErrorResult) results.get(3).results.get(0)).getErr());
+
+        // Check that the second operation failed after the first request
+        assertEquals(KeeperException.Code.NODEEXISTS.intValue(),
+                ((ErrorResult) results.get(1).results.get(1)).getErr());
+        assertEquals(KeeperException.Code.NODEEXISTS.intValue(),
+                ((ErrorResult) results.get(2).results.get(1)).getErr());
+        assertEquals(KeeperException.Code.NODEEXISTS.intValue(),
+                ((ErrorResult) results.get(3).results.get(1)).getErr());
+
+    }
+}
\ No newline at end of file

Propchange: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiAsyncTransactionTest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain