Author: camille
Date: Mon Jan 16 02:34:22 2012
New Revision: 1231810
URL: http://svn.apache.org/viewvc?rev=1231810&view=rev
Log:
ZOOKEEPER-1351. invalid test verification in MultiTransactionTest (phunt via camille)
Modified:
zookeeper/branches/branch-3.4/CHANGES.txt
zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java
Modified: zookeeper/branches/branch-3.4/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/CHANGES.txt?rev=1231810&r1=1231809&r2=1231810&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/CHANGES.txt (original)
+++ zookeeper/branches/branch-3.4/CHANGES.txt Mon Jan 16 02:34:22 2012
@@ -14,6 +14,8 @@ BUGFIXES:
ZOOKEEPER-1358. In StaticHostProviderTest.java, testNextDoesNotSleepForZero tests that
hostProvider.next(0)
doesn't sleep by checking that the latency of this call is less than 10sec (Alex Shraer
via camille)
+
+ ZOOKEEPER-1351. invalid test verification in MultiTransactionTest (phunt via camille)
Release 3.4.2 - 2011-12-21
Modified: zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java?rev=1231810&r1=1231809&r2=1231810&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java
(original)
+++ zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java
Mon Jan 16 02:34:22 2012
@@ -17,67 +17,47 @@
package org.apache.zookeeper.test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
import org.apache.log4j.Logger;
-import org.apache.zookeeper.*;
+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.OpResult.ErrorResult;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.server.SyncRequestProcessor;
-import org.apache.zookeeper.server.ZooKeeperServer;
-import org.apache.zookeeper.OpResult.ErrorResult;
-import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.ArrayList;
-
-import org.apache.zookeeper.data.Stat;
-
-import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
-
-public class MultiTransactionTest extends ZKTestCase implements Watcher {
+public class MultiTransactionTest extends ClientBase {
private static final Logger LOG = Logger.getLogger(MultiTransactionTest.class);
- private static final String HOSTPORT = "127.0.0.1:" + PortAssignment.unique();
-
private ZooKeeper zk;
- private ServerCnxnFactory serverFactory;
-
- @Override
- public void process(WatchedEvent event) {
- // ignore
- }
@Before
- public void setupZk() throws Exception {
- File tmpDir = ClientBase.createTmpDir();
- ClientBase.setupTestEnv();
- ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ public void setUp() throws Exception {
SyncRequestProcessor.setSnapCount(150);
- final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
- serverFactory = ServerCnxnFactory.createFactory(PORT, -1);
- serverFactory.startup(zks);
- LOG.info("starting up the zookeeper server .. waiting");
- Assert.assertTrue("waiting for server being up",
- ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
- zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
- }
-
- @After
- public void shutdownServer() throws Exception {
- zk.close();
- serverFactory.shutdown();
+ super.setUp();
+ zk = createClient();
}
@Test
public void testCreate() throws Exception {
- List<OpResult> results = new ArrayList<OpResult>();
-
- results = zk.multi(Arrays.asList(
+ 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)
@@ -225,6 +205,52 @@ public class MultiTransactionTest extend
}
}
+ @Test
+ public void testWatchesTriggered() throws KeeperException, InterruptedException {
+ HasTriggeredWatcher watcher = new HasTriggeredWatcher();
+ zk.getChildren("/", watcher);
+ zk.multi(Arrays.asList(
+ Op.create("/t", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
+ Op.delete("/t", -1)
+ ));
+ assertTrue(watcher.triggered.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS));
+ }
+
+ @Test
+ public void testNoWatchesTriggeredForFailedMultiRequest() throws InterruptedException,
KeeperException {
+ HasTriggeredWatcher watcher = new HasTriggeredWatcher();
+ zk.getChildren("/", watcher);
+ try {
+ zk.multi(Arrays.asList(
+ Op.create("/t", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
+ Op.delete("/nonexisting", -1)
+ ));
+ fail("expected previous multi op to fail!");
+ } catch (KeeperException.NoNodeException e) {
+ // expected
+ }
+ SyncCallback cb = new SyncCallback();
+ zk.sync("/", cb, null);
+
+ // by waiting for the callback we're assured that the event queue is flushed
+ cb.done.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS);
+ assertEquals(1, watcher.triggered.getCount());
+ }
+
+ private static class HasTriggeredWatcher implements Watcher {
+ private final CountDownLatch triggered = new CountDownLatch(1);
+ @Override
+ public void process(WatchedEvent event) {
+ triggered.countDown();
+ }
+ }
+ private static class SyncCallback implements AsyncCallback.VoidCallback {
+ private final CountDownLatch done = new CountDownLatch(1);
+ @Override
+ public void processResult(int rc, String path, Object ctx) {
+ done.countDown();
+ }
+ }
}
|