You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ma...@apache.org on 2009/06/25 00:59:34 UTC

svn commit: r788205 [2/2] - in /hadoop/zookeeper/trunk: ./ docs/ src/c/src/ src/c/tests/ src/docs/src/documentation/content/xdocs/ src/java/main/org/apache/zookeeper/ src/java/test/org/apache/zookeeper/test/

Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootAsyncTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootAsyncTest.java?rev=788205&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootAsyncTest.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootAsyncTest.java Wed Jun 24 22:59:34 2009
@@ -0,0 +1,48 @@
+/**
+ * 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 org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.junit.Before;
+
+public class ChrootAsyncTest extends AsyncOpsTest {
+    private static final Logger LOG = Logger.getLogger(ChrootAsyncTest.class);
+
+    @Before
+    @Override
+    protected void setUp() throws Exception {
+        String hp = hostPort;
+        hostPort = hostPort + "/chrootasynctest";
+
+        super.setUp();
+
+        LOG.info("STARTING " + getName());
+
+        ZooKeeper zk = createClient(hp);
+        try {
+            zk.create("/chrootasynctest", null, Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
+        } finally {
+            zk.close();
+        }
+    }
+}
\ No newline at end of file

Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootClientTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootClientTest.java?rev=788205&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootClientTest.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootClientTest.java Wed Jun 24 22:59:34 2009
@@ -0,0 +1,48 @@
+/**
+ * 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 org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.junit.Before;
+
+public class ChrootClientTest extends ClientTest {
+    private static final Logger LOG = Logger.getLogger(ChrootClientTest.class);
+
+    @Before
+    @Override
+    protected void setUp() throws Exception {
+        String hp = hostPort;
+        hostPort = hostPort + "/chrootclienttest";
+
+        super.setUp();
+
+        LOG.info("STARTING " + getName());
+
+        ZooKeeper zk = createClient(hp);
+        try {
+            zk.create("/chrootclienttest", null, Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
+        } finally {
+            zk.close();
+        }
+    }
+}
\ No newline at end of file

Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootTest.java?rev=788205&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootTest.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootTest.java Wed Jun 24 22:59:34 2009
@@ -0,0 +1,132 @@
+/**
+ * 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 java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.test.AsyncOps.ACLCB;
+import org.apache.zookeeper.test.AsyncOps.ChildrenCB;
+import org.apache.zookeeper.test.AsyncOps.DataCB;
+import org.apache.zookeeper.test.AsyncOps.StatCB;
+import org.apache.zookeeper.test.AsyncOps.StringCB;
+import org.apache.zookeeper.test.AsyncOps.VoidCB;
+import org.junit.Test;
+
+public class ChrootTest extends ClientBase {
+    private class MyWatcher implements Watcher {
+        private final String path;
+        private String eventPath;
+        private CountDownLatch latch = new CountDownLatch(1);
+
+        public MyWatcher(String path) {
+            this.path = path;
+        }
+        public void process(WatchedEvent event) {
+            System.out.println("latch:" + path + " " + event.getPath());
+            this.eventPath = event.getPath();
+            latch.countDown();
+        }
+        public boolean matches() throws InterruptedException {
+            if (!latch.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)) {
+                fail("No watch received within timeout period " + path);
+            }
+            return path.equals(eventPath);
+        }
+    }
+
+    public void testChrootSynchronous()
+        throws IOException, InterruptedException, KeeperException
+    {
+        ZooKeeper zk1 = createClient();
+        try {
+            zk1.create("/ch1", null, Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
+        } finally {
+            if(zk1 != null)
+                zk1.close();
+        }
+        ZooKeeper zk2 = createClient(hostPort + "/ch1");
+        try {
+            assertEquals("/ch2",
+                    zk2.create("/ch2", null, Ids.OPEN_ACL_UNSAFE,
+                            CreateMode.PERSISTENT));
+        } finally {
+            if(zk2 != null)
+                zk2.close();
+        }
+
+        zk1 = createClient();
+        zk2 = createClient(hostPort + "/ch1");
+        try {
+            // check get
+            MyWatcher w1 = new MyWatcher("/ch1");
+            assertNotNull(zk1.exists("/ch1", w1));
+            MyWatcher w2 = new MyWatcher("/ch1/ch2");
+            assertNotNull(zk1.exists("/ch1/ch2", w2));
+
+            MyWatcher w3 = new MyWatcher("/ch2");
+            assertNotNull(zk2.exists("/ch2", w3));
+
+            // check set
+            zk1.setData("/ch1", "1".getBytes(), -1);
+            zk2.setData("/ch2", "2".getBytes(), -1);
+
+            // check watches
+            assertTrue(w1.matches());
+            assertTrue(w2.matches());
+            assertTrue(w3.matches());
+
+            // check exceptions
+            try {
+                zk2.setData("/ch3", "3".getBytes(), -1);
+            } catch (KeeperException.NoNodeException e) {
+                assertEquals("/ch3", e.getPath());
+            }
+
+            assertTrue(Arrays.equals("1".getBytes(),
+                    zk1.getData("/ch1", false, null)));
+            assertTrue(Arrays.equals("2".getBytes(),
+                    zk1.getData("/ch1/ch2", false, null)));
+            assertTrue(Arrays.equals("2".getBytes(),
+                    zk2.getData("/ch2", false, null)));
+
+            // check delete
+            zk2.delete("/ch2", -1);
+            zk1.delete("/ch1", -1);
+            assertNull(zk1.exists("/ch1", false));
+            assertNull(zk1.exists("/ch1/ch2", false));
+            assertNull(zk2.exists("/ch2", false));
+        } finally {
+            if(zk1 != null)
+                zk1.close();
+            if(zk2 != null)
+                zk2.close();
+        }
+    }
+}
\ No newline at end of file

Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java?rev=788205&r1=788204&r2=788205&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java Wed Jun 24 22:59:34 2009
@@ -254,13 +254,21 @@
         
         return tmpDir;
     }
-        
+    private static int getPort(String hostPort) {
+        String portstr = hostPort.split(":")[1];
+        String[] pc = portstr.split("/");
+        if (pc.length > 1) {
+            portstr = pc[0];
+        }
+        return Integer.parseInt(portstr);
+    }
+    
     static NIOServerCnxn.Factory createNewServerInstance(File dataDir,
             NIOServerCnxn.Factory factory, String hostPort, int maxCnxns)
         throws IOException, InterruptedException 
     {
         ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
-        final int PORT = Integer.parseInt(hostPort.split(":")[1]);
+        final int PORT = getPort(hostPort);
         if (factory == null) {
             factory = new NIOServerCnxn.Factory(PORT,maxCnxns);
         }
@@ -278,7 +286,7 @@
     {
         if (factory != null) {
             factory.shutdown();
-            final int PORT = Integer.parseInt(hostPort.split(":")[1]);
+            final int PORT = getPort(hostPort);
 
             assertTrue("waiting for server down",
                        ClientBase.waitForServerDown("127.0.0.1:" + PORT,

Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java?rev=788205&r1=788204&r2=788205&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java Wed Jun 24 22:59:34 2009
@@ -287,9 +287,9 @@
         try {
             MyWatcher watcher = new MyWatcher();
             zk = createClient(watcher, hostPort);
-            //LOG.info("Created client: " + zk.describeCNXN());
             LOG.info("Before create /benwashere");
-            zk.create("/benwashere", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zk.create("/benwashere", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
             LOG.info("After create /benwashere");
             try {
                 zk.setData("/benwashere", "hi".getBytes(), 57);
@@ -301,7 +301,7 @@
             }
             LOG.info("Before delete /benwashere");
             zk.delete("/benwashere", 0);
-            LOG.info("Before delete /benwashere");
+            LOG.info("After delete /benwashere");
             zk.close();
             //LOG.info("Closed client: " + zk.describeCNXN());
             Thread.sleep(2000);
@@ -318,13 +318,16 @@
             }
             Stat stat = new Stat();
             // Test basic create, ls, and getData
+            zk.create("/pat", "Pat was here".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
             LOG.info("Before create /ben");
-            zk.create("/ben", "Ben was here".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-            LOG.info("Before getChildren /");
-            List<String> children = zk.getChildren("/", false);
-            assertEquals(2, children.size());
-            assertEquals("ben", children.get(1));
-            String value = new String(zk.getData("/ben", false, stat));
+            zk.create("/pat/ben", "Ben was here".getBytes(),
+                    Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            LOG.info("Before getChildren /pat");
+            List<String> children = zk.getChildren("/pat", false);
+            assertEquals(1, children.size());
+            assertEquals("ben", children.get(0));
+            String value = new String(zk.getData("/pat/ben", false, stat));
             assertEquals("Ben was here", value);
             // Test stat and watch of non existent node
 
@@ -348,12 +351,12 @@
             assertEquals(EventType.NodeCreated, event.getType());
             assertEquals(KeeperState.SyncConnected, event.getState());
             // Test child watch and create with sequence
-            zk.getChildren("/ben", true);
+            zk.getChildren("/pat/ben", true);
             for (int i = 0; i < 10; i++) {
-                zk.create("/ben/" + i + "-", Integer.toString(i).getBytes(),
+                zk.create("/pat/ben/" + i + "-", Integer.toString(i).getBytes(),
                         Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL);
             }
-            children = zk.getChildren("/ben", false);
+            children = zk.getChildren("/pat/ben", false);
             Collections.sort(children);
             assertEquals(10, children.size());
             for (int i = 0; i < 10; i++) {
@@ -361,39 +364,43 @@
                 assertTrue("starts with -", name.startsWith(i + "-"));
                 byte b[];
                 if (withWatcherObj) {
-                    b = zk.getData("/ben/" + name, watcher, stat);
+                    b = zk.getData("/pat/ben/" + name, watcher, stat);
                 } else {
-                    b = zk.getData("/ben/" + name, true, stat);
+                    b = zk.getData("/pat/ben/" + name, true, stat);
                 }
                 assertEquals(Integer.toString(i), new String(b));
-                zk.setData("/ben/" + name, "new".getBytes(), stat.getVersion());
+                zk.setData("/pat/ben/" + name, "new".getBytes(),
+                        stat.getVersion());
                 if (withWatcherObj) {
-                    stat = zk.exists("/ben/" + name, watcher);
+                    stat = zk.exists("/pat/ben/" + name, watcher);
                 } else {
-                stat = zk.exists("/ben/" + name, true);
+                stat = zk.exists("/pat/ben/" + name, true);
                 }
-                zk.delete("/ben/" + name, stat.getVersion());
+                zk.delete("/pat/ben/" + name, stat.getVersion());
             }
             event = watcher.events.poll(10, TimeUnit.SECONDS);
-            assertEquals("/ben", event.getPath());
+            assertEquals("/pat/ben", event.getPath());
             assertEquals(EventType.NodeChildrenChanged, event.getType());
             assertEquals(KeeperState.SyncConnected, event.getState());
             for (int i = 0; i < 10; i++) {
                 event = watcher.events.poll(10, TimeUnit.SECONDS);
                 final String name = children.get(i);
-                assertEquals("/ben/" + name, event.getPath());
+                assertEquals("/pat/ben/" + name, event.getPath());
                 assertEquals(EventType.NodeDataChanged, event.getType());
                 assertEquals(KeeperState.SyncConnected, event.getState());
                 event = watcher.events.poll(10, TimeUnit.SECONDS);
-                assertEquals("/ben/" + name, event.getPath());
+                assertEquals("/pat/ben/" + name, event.getPath());
                 assertEquals(EventType.NodeDeleted, event.getType());
                 assertEquals(KeeperState.SyncConnected, event.getState());
             }
-            zk.create("/good\u0040path", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zk.create("/good\u0040path", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
 
-            zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
             try {
-                zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+                zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                        CreateMode.PERSISTENT);
                 fail("duplicate create allowed");
             } catch(KeeperException.NodeExistsException e) {
                 // OK, expected that