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 2011/08/14 18:40:04 UTC

svn commit: r1157559 - in /zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/ src/java/test/org/apache/zookeeper/

Author: mahadev
Date: Sun Aug 14 16:40:03 2011
New Revision: 1157559

URL: http://svn.apache.org/viewvc?rev=1157559&view=rev
Log:
ZOOKEEPER-839. deleteRecursive does not belong to the other methods. (mahadev)

Added:
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ZKUtil.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1157559&r1=1157558&r2=1157559&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Sun Aug 14 16:40:03 2011
@@ -276,6 +276,9 @@ BUGFIXES: 
   
   ZOOKEEPER-1144. ZooKeeperServer not starting on leader due to a race condition (Vishal K via camille)
 
+  ZOOKEEPER-839. deleteRecursive does not belong to the other methods.
+   (mahadev)
+
 IMPROVEMENTS:
   ZOOKEEPER-724. Improve junit test integration - log harness information 
   (phunt via mahadev)

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZKUtil.java?rev=1157559&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ZKUtil.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ZKUtil.java Sun Aug 14 16:40:03 2011
@@ -0,0 +1,124 @@
+/**
+ * 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.util.ArrayList;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.zookeeper.AsyncCallback.VoidCallback;
+import org.apache.zookeeper.common.PathUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+         
+public class ZKUtil {
+    private static final Logger LOG = LoggerFactory.getLogger(ZKUtil.class);
+    /**
+     * 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 static void deleteRecursive(ZooKeeper zk, final String pathRoot)
+        throws InterruptedException, KeeperException
+    {
+        PathUtils.validatePath(pathRoot);
+      
+        List<String> tree = listSubTreeBFS(zk, 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
+            zk.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>
+     * @param zk the zookeeper handle
+     * @param pathRoot the path to be deleted
+     * @param cb call back method
+     * @param ctx the context the callback method is called with
+     * @throws IllegalArgumentException if an invalid path is specified
+     */
+    public static void deleteRecursive(ZooKeeper zk, final String pathRoot, VoidCallback cb,
+        Object ctx)
+        throws InterruptedException, KeeperException
+    {
+        PathUtils.validatePath(pathRoot);
+      
+        List<String> tree = listSubTreeBFS(zk, 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
+            zk.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 zk the zookeeper handle
+     * @param pathRoot The znode path, for which the entire subtree needs to be listed.
+     * @throws InterruptedException 
+     * @throws KeeperException 
+     */
+    public static List<String> listSubTreeBFS(ZooKeeper zk, 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 = zk.getChildren(node, false);
+            for (final String child : children) {
+                final String childPath = node + "/" + child;
+                queue.add(childPath);
+                tree.add(childPath);
+            }
+        }
+        return tree;
+    }
+    
+}
\ No newline at end of file

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=1157559&r1=1157558&r2=1157559&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Sun Aug 14 16:40:03 2011
@@ -920,90 +920,7 @@ public class ZooKeeper {
         return new Transaction(this);
     }
 
-    /**
-     * 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;
-    }
-    
+  
 
     
     /**

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java?rev=1157559&r1=1157558&r2=1157559&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java Sun Aug 14 16:40:03 2011
@@ -690,7 +690,7 @@ public class ZooKeeperMain {
             zk.delete(path, watch ? Integer.parseInt(args[2]) : -1);
         } else if (cmd.equals("rmr") && args.length >= 2) {
             path = args[1];
-            zk.deleteRecursive(path);
+            ZKUtil.deleteRecursive(zk, path);
         } else if (cmd.equals("set") && args.length >= 3) {
             path = args[1];
             stat = zk.setData(path, args[2].getBytes(),

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java?rev=1157559&r1=1157558&r2=1157559&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java Sun Aug 14 16:40:03 2011
@@ -65,7 +65,7 @@ public class ZooKeeperTest extends Clien
         Assert.assertTrue(children.contains("b"));
         Assert.assertTrue(children.contains("c"));
 
-        zk.deleteRecursive("/a");
+        ZKUtil.deleteRecursive(zk, "/a");
         Assert.assertNull(zk.exists("/a", null));
     }
 
@@ -116,7 +116,7 @@ public class ZooKeeperTest extends Clien
 
         };
         final AtomicInteger ctx = new AtomicInteger(3);
-        zk.deleteRecursive("/a", cb, ctx);
+        ZKUtil.deleteRecursive(zk, "/a", cb, ctx);
         synchronized (ctx) {
             ctx.wait();
         }