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