Author: henry
Date: Thu Apr 15 07:23:44 2010
New Revision: 934312
URL: http://svn.apache.org/viewvc?rev=934312&view=rev
Log:
ZOOKEEPER-729. Java client API to recursively delete a subtree. (Kay Kay via henry)
Added:
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=934312&r1=934311&r2=934312&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Thu Apr 15 07:23:44 2010
@@ -23,7 +23,8 @@ IMPROVEMENTS:
(phunt via mahadev)
NEW FEATURES:
-
+ ZOOKEEPER-729. Java client API to recursively delete a subtree. (Kay
+ Kay via henry)
Release 3.3.0 - 2010-03-24
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=934312&r1=934311&r2=934312&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Thu Apr 15 07:23:44
2010
@@ -21,8 +21,10 @@ package org.apache.zookeeper;
import java.io.IOException;
import java.net.SocketAddress;
import java.util.ArrayList;
+import java.util.Deque;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -730,6 +732,93 @@ public class ZooKeeper {
}
}
+
+ /**
+ * Recursively delete the node with the given path.
+ * <p>
+ * Important: All versions, of all nodes, under the given node are deleted.
+ * <p>
+ * If there is an error with deleting one of the sub-nodes in the tree,
+ * this operation would abort and would be the responsibility of the app to handle the
same.
+ *
+ * See {@link #delete(String, int)} for more details.
+ *
+ * @throws IllegalArgumentException if an invalid path is specified
+ */
+ public void deleteRecursive(final String pathRoot)
+ throws InterruptedException, KeeperException
+ {
+ PathUtils.validatePath(pathRoot);
+
+ List<String> tree = this.listSubTreeBFS(pathRoot);
+ LOG.debug("Deleting " + tree);
+ LOG.debug("Deleting " + tree.size() + " subnodes ");
+ for (int i = tree.size() - 1; i >= 0 ; --i) {
+ //Delete the leaves first and eventually get rid of the root
+ this.delete(tree.get(i), -1); //Delete all versions of the node with -1.
+ }
+ }
+
+
+ /**
+ * Recursively delete the node with the given path. (async version).
+ *
+ * <p>
+ * Important: All versions, of all nodes, under the given node are deleted.
+ * <p>
+ * If there is an error with deleting one of the sub-nodes in the tree,
+ * this operation would abort and would be the responsibility of the app to handle the
same.
+ * <p>
+ *
+ * @throws IllegalArgumentException if an invalid path is specified
+ */
+ public void deleteRecursive(final String pathRoot, VoidCallback cb,
+ Object ctx)
+ throws InterruptedException, KeeperException
+ {
+ PathUtils.validatePath(pathRoot);
+
+ List<String> tree = this.listSubTreeBFS(pathRoot);
+ LOG.debug("Deleting " + tree);
+ LOG.debug("Deleting " + tree.size() + " subnodes ");
+ for (int i = tree.size() - 1; i >= 0 ; --i) {
+ //Delete the leaves first and eventually get rid of the root
+ this.delete(tree.get(i), -1, cb, ctx); //Delete all versions of the node with
-1.
+ }
+ }
+
+ /**
+ * BFS Traversal of the system under pathRoot, with the entries in the list, in the same
order as that of the traversal.
+ * <p>
+ * <b>Important:</b> This is <i>not an atomic snapshot</i> of
the tree ever, but the state as it exists across multiple RPCs from zkClient to the ensemble.
+ * For practical purposes, it is suggested to bring the clients to the ensemble down
(i.e. prevent writes to pathRoot) to 'simulate' a snapshot behavior.
+ *
+ * @param pathRoot The znode path, for which the entire subtree needs to be listed.
+ * @throws InterruptedException
+ * @throws KeeperException
+ */
+ public List<String> listSubTreeBFS(final String pathRoot) throws KeeperException,
InterruptedException {
+ Deque<String> queue = new LinkedList<String>();
+ List<String> tree = new ArrayList<String>();
+ queue.add(pathRoot);
+ tree.add(pathRoot);
+ while (true) {
+ String node = queue.pollFirst();
+ if (node == null) {
+ break;
+ }
+ List<String> children = this.getChildren(node, false);
+ for (final String child : children) {
+ final String childPath = node + "/" + child;
+ queue.add(childPath);
+ tree.add(childPath);
+ }
+ }
+ return tree;
+ }
+
+
+
/**
* The Asynchronous version of delete. The request doesn't actually until
* the asynchronous callback is called.
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java?rev=934312&r1=934311&r2=934312&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java Thu Apr 15
07:23:44 2010
@@ -65,6 +65,7 @@ public class ZooKeeperMain {
commandMap.put("close","");
commandMap.put("create", "[-s] [-e] path data acl");
commandMap.put("delete","path [version]");
+ commandMap.put("rmr","path");
commandMap.put("set","path data [version]");
commandMap.put("get","path [watch]");
commandMap.put("ls","path [watch]");
@@ -681,6 +682,9 @@ public class ZooKeeperMain {
} else if (cmd.equals("delete") && args.length >= 2) {
path = args[1];
zk.delete(path, watch ? Integer.parseInt(args[2]) : -1);
+ } else if (cmd.equals("rmr") && args.length >= 2) {
+ path = args[1];
+ zk.deleteRecursive(path);
} else if (cmd.equals("set") && args.length >= 3) {
path = args[1];
stat = zk.setData(path, args[2].getBytes(),
Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java?rev=934312&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java Thu Apr 15
07:23:44 2010
@@ -0,0 +1,125 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.zookeeper.AsyncCallback.VoidCallback;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ *
+ * Testing Zookeeper public methods
+ *
+ */
+public class ZooKeeperTest extends ClientBase {
+
+ @Test
+ public void testDeleteRecursive() throws IOException, InterruptedException,
+ KeeperException {
+ final ZooKeeper zk = createClient();
+ // making sure setdata works on /
+ zk.setData("/", "some".getBytes(), -1);
+ zk.create("/a", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/b", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/b/v", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/b/v/1", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/c", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/c/v", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ List<String> children = zk.getChildren("/a", false);
+
+ Assert.assertEquals("2 children - b & c should be present ", children
+ .size(), 2);
+ Assert.assertTrue(children.contains("b"));
+ Assert.assertTrue(children.contains("c"));
+
+ zk.deleteRecursive("/a");
+ Assert.assertNull(zk.exists("/a", null));
+ }
+
+ @Test
+ public void testDeleteRecursiveAsync() throws IOException,
+ InterruptedException, KeeperException {
+ final ZooKeeper zk = createClient();
+ // making sure setdata works on /
+ zk.setData("/", "some".getBytes(), -1);
+ zk.create("/a", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/b", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/b/v", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/b/v/1", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/c", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ zk.create("/a/c/v", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+
+ for (int i = 0; i < 50; ++i) {
+ zk.create("/a/c/" + i, "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ }
+ List<String> children = zk.getChildren("/a", false);
+
+ Assert.assertEquals("2 children - b & c should be present ", children
+ .size(), 2);
+ Assert.assertTrue(children.contains("b"));
+ Assert.assertTrue(children.contains("c"));
+
+ VoidCallback cb = new VoidCallback() {
+
+ @Override
+ public void processResult(int rc, String path, Object ctx) {
+ synchronized (ctx) {
+ ((AtomicInteger) ctx).set(4);
+ ctx.notify();
+ }
+ }
+
+ };
+ final AtomicInteger ctx = new AtomicInteger(3);
+ zk.deleteRecursive("/a", cb, ctx);
+ synchronized (ctx) {
+ ctx.wait();
+ }
+ Assert.assertEquals(4, ((AtomicInteger) ctx).get());
+ }
+}
|