zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [zookeeper] enixon commented on a change in pull request #899: ZOOKEEPER-3354: Improve efficiency of DeleteAllCommand
Date Fri, 03 May 2019 19:22:51 GMT
enixon commented on a change in pull request #899: ZOOKEEPER-3354: Improve efficiency of DeleteAllCommand
URL: https://github.com/apache/zookeeper/pull/899#discussion_r280899376
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ZKUtil.java
 ##########
 @@ -45,20 +48,67 @@
      *
      * @throws IllegalArgumentException if an invalid path is specified
      */
-    public static void deleteRecursive(ZooKeeper zk, final String pathRoot)
+    public static boolean 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.
+
+        int asyncReqRateLimit = 10;
+        // Try deleting the tree nodes in batches of size 1000.
+        // If some batch failed, try again with batches of size 1 to delete as
+        // many nodes as possible.
+        boolean success = deleteInBatch(zk, tree, 1000, asyncReqRateLimit);
+        if (!success) {
+            LOG.debug("Failed to delete all nodes in batches of 1000.");
+            LOG.debug("Retry with batches of size 1...");
+            success = deleteInBatch(zk, tree, 1, asyncReqRateLimit);
+        }
+        return success;
+    }
+
+    private static class BatchedDeleteCbContext {
+        public Semaphore sem;
+        public AtomicBoolean success;
+
+        public BatchedDeleteCbContext(int rateLimit) {
+            sem = new Semaphore(rateLimit);
+            success = new AtomicBoolean(true);
         }
     }
 
+    private static boolean deleteInBatch(ZooKeeper zk, List<String> tree,
+        int batchSize, int rateLimit)
+        throws InterruptedException
+    {
+        List<Op> ops = new ArrayList<>();
+        BatchedDeleteCbContext context = new BatchedDeleteCbContext(rateLimit);
+        MultiCallback cb = (rc, path, ctx, opResults) -> {
+            ((BatchedDeleteCbContext)ctx).sem.release();
+            if (rc != Code.OK.intValue()) {
+                ((BatchedDeleteCbContext)ctx).success.set(false);
 
 Review comment:
   This is certainly possible (and I just did) but the desirability of a fail-fast depends
on the reason that the delete failed. If the problem applies to a small subset of the target
nodes of the subtree then it strikes me as best to honor the original command intent and at
least attempt all the deletes to get as many as possible to succeed. If the problem applies
to many of the targets then better not to waste the user's time.
   
   I've included a new test to demonstrate such a scenario.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message