You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/11/10 07:08:44 UTC

[GitHub] [pulsar] wolfstudy opened a new pull request #12711: Fix znode leakage caused by deleting tenant

wolfstudy opened a new pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711


   Signed-off-by: xiaolongran <xi...@tencent.com>
   
   
   Fixes #12710 
   
   
   ### Motivation
   
   According to the previous code logic, if the tenant resource is not forcibly deleted, the zk-node resource will be leaked, because under the condition of **!force**, the `clearTenantPersistence` code logic will not be called.
   
   ```
       protected void internalDeleteTenant(AsyncResponse asyncResponse, String tenant, boolean force) {
           if (force) {
               internalDeleteTenantForcefully(asyncResponse, tenant);
           } else {
               internalDeleteTenant(asyncResponse, tenant);
           }
       }
   ```
   
   
   
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy commented on pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#issuecomment-966022838


   /pulsarbot run-failure-checks


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r748686772



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java
##########
@@ -181,19 +182,46 @@ protected static String joinPath(String... parts) {
         return sb.toString();
     }
 
-
-
     protected static void deleteRecursive(BaseResources resources, final String pathRoot) throws MetadataStoreException {
         PathUtils.validatePath(pathRoot);
         List<String> tree = listSubTreeBFS(resources, pathRoot);
         log.debug("Deleting {} with size {}", tree, tree.size());
-        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
             resources.delete(tree.get(i));
         }
     }
 
+    protected static CompletableFuture<Void> deleteRecursiveAsync(BaseResources resources, final String pathRoot) {
+        PathUtils.validatePath(pathRoot);
+        List<String> tree = null;
+        try {
+            tree = listSubTreeBFS(resources, pathRoot);
+        } catch (MetadataStoreException e) {
+            // no-op
+        }
+
+        if (tree != null) {
+            log.debug("Deleting {} with size {}", tree, tree.size());
+
+            final List<CompletableFuture<Void>> futures = new ArrayList<>();
+            for (int i = tree.size() - 1; i >= 0; --i) {
+                // Delete the leaves first and eventually get rid of the root
+                futures.add(resources.deleteAsync(tree.get(i)));
+            }
+
+            FutureUtil.waitForAll(futures).handle((result, exception) -> {
+                if (exception != null) {
+                    log.error("Failed to remove partitioned topics", exception);
+                    return null;
+                }
+                return null;
+            });
+        }
+
+        return null;

Review comment:
       Ok, will fix this




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] gaoran10 commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r747633308



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java
##########
@@ -194,6 +197,38 @@ protected static void deleteRecursive(BaseResources resources, final String path
         }
     }
 
+    protected static CompletableFuture<Void> deleteRecursiveAsync(BaseResources resources, final String pathRoot) {
+        PathUtils.validatePath(pathRoot);
+        List<String> tree = null;
+        try {
+            tree = listSubTreeBFS(resources, pathRoot);
+        } catch (MetadataStoreException e) {
+

Review comment:
       We could ignore this exception, right? Could you add some explanation?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] gaoran10 commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r747633513



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java
##########
@@ -194,6 +197,38 @@ protected static void deleteRecursive(BaseResources resources, final String path
         }
     }
 
+    protected static CompletableFuture<Void> deleteRecursiveAsync(BaseResources resources, final String pathRoot) {
+        PathUtils.validatePath(pathRoot);
+        List<String> tree = null;
+        try {
+            tree = listSubTreeBFS(resources, pathRoot);
+        } catch (MetadataStoreException e) {
+
+        }
+
+        if (tree != null) {
+            log.debug("Deleting {} with size {}", tree, tree.size());
+            log.debug("Deleting " + tree.size() + " subnodes ");

Review comment:
       Please check this log.

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
##########
@@ -484,42 +474,39 @@ protected void internalDeleteNamespaceForcefully(AsyncResponse asyncResponse, bo
                     asyncResponse.resume(new RestException((PulsarAdminException) exception.getCause()));
                     return null;
                 } else {
-                    log.error("[{}] Failed to remove owned namespace {}", clientAppId(), namespaceName, exception);
+                    log.error("[{}] Failed to remove forcefully owned namespace {}",
+                            clientAppId(), namespaceName, exception);
                     asyncResponse.resume(new RestException(exception.getCause()));
                     return null;
                 }
             }
 
-            try {
-                // remove partitioned topics znode
-                pulsar().getPulsarResources().getNamespaceResources().getPartitionedTopicResources()
-                        .clearPartitionedTopicMetadata(namespaceName);
-
-                try {
-                    pulsar().getPulsarResources().getTopicResources().clearDomainPersistence(namespaceName).get();
-                    pulsar().getPulsarResources().getTopicResources().clearNamespacePersistence(namespaceName).get();
-                } catch (ExecutionException | InterruptedException e) {
-                    // warn level log here since this failure has no side effect besides left a un-used metadata
-                    // and also will not affect the re-creation of namespace
-                    log.warn("[{}] Failed to remove managed-ledger for {}", clientAppId(), namespaceName, e);
-                }
-
-                // we have successfully removed all the ownership for the namespace, the policies znode can be deleted
-                // now
-                namespaceResources().deletePolicies(namespaceName);
-
-                try {
-                    getLocalPolicies().deleteLocalPolicies(namespaceName);
-                } catch (NotFoundException nne) {
-                    // If the z-node with the modified information is not there anymore, we're already good
-                }
-            } catch (Exception e) {
-                log.error("[{}] Failed to remove owned namespace {} from ZK", clientAppId(), namespaceName, e);
-                asyncResponse.resume(new RestException(e));
-                return null;
-            }
+            // clear resource of `/namespace/{namespaceName}` for zk-node
+            namespaceResources().deleteNamespaceAsync(namespaceName)
+                    .thenCompose(ignore -> namespaceResources().getPartitionedTopicResources()
+                            .clearPartitionedTopicMetadataAsync(namespaceName))
+                    // clear resource for manager-ledger z-node
+                    .thenCompose(ignore -> pulsar().getPulsarResources().getTopicResources()
+                            .clearDomainPersistence(namespaceName))
+                    .thenCompose(ignore -> pulsar().getPulsarResources().getTopicResources()
+                            .clearNamespacePersistence(namespaceName))
+                    // we have successfully removed all the ownership for the namespace, the policies
+                    // z-node can be deleted now
+                    .thenCompose(ignore -> namespaceResources().deletePoliciesAsync(namespaceName))
+                    // clear z-node of local policies
+                    .thenCompose(ignore -> getLocalPolicies().deleteLocalPoliciesAsync(namespaceName))
+                    .whenComplete((ignore, ex) -> {
+                        if (ex != null) {
+                            log.warn("[{}] Failed to force remove namespace or managed-ledger for {}",
+                                    clientAppId(), namespaceName, ex);
+                            asyncResponse.resume(new RestException(ex));
+                        } else {
+                            log.info("[{}] Remove forcefully namespace or managed-ledger successfully {}",
+                                    clientAppId(), namespaceName);
+                            asyncResponse.resume(Response.noContent().build());
+                        }
+                    });
 

Review comment:
       It seems that this code block was presented twice, could we add a method for this code block and reuse it.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java
##########
@@ -194,6 +197,38 @@ protected static void deleteRecursive(BaseResources resources, final String path
         }
     }
 
+    protected static CompletableFuture<Void> deleteRecursiveAsync(BaseResources resources, final String pathRoot) {
+        PathUtils.validatePath(pathRoot);
+        List<String> tree = null;
+        try {
+            tree = listSubTreeBFS(resources, pathRoot);
+        } catch (MetadataStoreException e) {
+
+        }
+
+        if (tree != null) {
+            log.debug("Deleting {} with size {}", tree, tree.size());
+            log.debug("Deleting " + tree.size() + " subnodes ");
+
+            final List<CompletableFuture<Void>> futures = new ArrayList<>();
+            for (int i = tree.size() - 1; i >= 0; --i) {
+                // Delete the leaves first and eventually get rid of the root
+                futures.add(resources.deleteAsync(tree.get(i)));
+            }
+
+            FutureUtil.waitForAll(futures).handle((result, exception) -> {
+                if (exception != null) {
+                    log.error("Failed to remove partitioned topics", exception);
+                    return null;
+                }
+                Response.noContent().build();

Review comment:
       It seems that this statement is useless.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r748360066



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java
##########
@@ -215,6 +258,17 @@ public boolean partitionedTopicExists(TopicName tn) throws MetadataStoreExceptio
                     tn.getEncodedLocalName()));
         }
 
+        public CompletableFuture<Void> clearPartitionedTopicMetadataAsync(NamespaceName namespaceName) {
+            final String globalPartitionedPath = joinPath(PARTITIONED_TOPIC_PATH, namespaceName.toString());
+
+            return existsAsync(globalPartitionedPath).thenAccept(exist -> {

Review comment:
       ```suggestion
               return existsAsync(globalPartitionedPath).thenCompose(exist -> {
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] gaoran10 commented on pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#issuecomment-964976148


   Hi, @wolfstudy, could you add a unit test for this PR?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r747940904



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java
##########
@@ -194,6 +197,38 @@ protected static void deleteRecursive(BaseResources resources, final String path
         }
     }
 
+    protected static CompletableFuture<Void> deleteRecursiveAsync(BaseResources resources, final String pathRoot) {
+        PathUtils.validatePath(pathRoot);
+        List<String> tree = null;
+        try {
+            tree = listSubTreeBFS(resources, pathRoot);
+        } catch (MetadataStoreException e) {
+
+        }
+
+        if (tree != null) {
+            log.debug("Deleting {} with size {}", tree, tree.size());
+            log.debug("Deleting " + tree.size() + " subnodes ");
+
+            final List<CompletableFuture<Void>> futures = new ArrayList<>();
+            for (int i = tree.size() - 1; i >= 0; --i) {
+                // Delete the leaves first and eventually get rid of the root
+                futures.add(resources.deleteAsync(tree.get(i)));
+            }
+
+            FutureUtil.waitForAll(futures).handle((result, exception) -> {
+                if (exception != null) {
+                    log.error("Failed to remove partitioned topics", exception);
+                    return null;
+                }
+                Response.noContent().build();

Review comment:
       yes, will remove this

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java
##########
@@ -194,6 +197,38 @@ protected static void deleteRecursive(BaseResources resources, final String path
         }
     }
 
+    protected static CompletableFuture<Void> deleteRecursiveAsync(BaseResources resources, final String pathRoot) {
+        PathUtils.validatePath(pathRoot);
+        List<String> tree = null;
+        try {
+            tree = listSubTreeBFS(resources, pathRoot);
+        } catch (MetadataStoreException e) {
+
+        }
+
+        if (tree != null) {
+            log.debug("Deleting {} with size {}", tree, tree.size());
+            log.debug("Deleting " + tree.size() + " subnodes ");

Review comment:
       Ok, will delete it




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r749980874



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java
##########
@@ -215,12 +258,27 @@ public boolean partitionedTopicExists(TopicName tn) throws MetadataStoreExceptio
                     tn.getEncodedLocalName()));
         }
 
-        public void clearPartitionedTopicMetadata(NamespaceName namespaceName) throws MetadataStoreException {
+        public CompletableFuture<Void> clearPartitionedTopicMetadataAsync(NamespaceName namespaceName) {
             final String globalPartitionedPath = joinPath(PARTITIONED_TOPIC_PATH, namespaceName.toString());
-            // check whether partitioned topics metadata node exist
-            if (exists(globalPartitionedPath)) {
-                deleteRecursive(this, globalPartitionedPath);
-            }
+
+            CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+
+            deleteRecursiveAsync(this, globalPartitionedPath)
+                    .thenAccept(ignore -> {
+                        log.info("Clear partitioned topic metadata [{}] success.", namespaceName);
+                        completableFuture.complete(null);
+                    }).exceptionally(ex -> {
+                if (ex.getCause().getCause() instanceof KeeperException) {

Review comment:
       ok, will fix this




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy commented on pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#issuecomment-965098263


   > @wolfstudy Please help add a test to avoid the regression.
   
   ping @codelipenghui PTAL again, thanks


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r748686193



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java
##########
@@ -215,6 +258,17 @@ public boolean partitionedTopicExists(TopicName tn) throws MetadataStoreExceptio
                     tn.getEncodedLocalName()));
         }
 
+        public CompletableFuture<Void> clearPartitionedTopicMetadataAsync(NamespaceName namespaceName) {
+            final String globalPartitionedPath = joinPath(PARTITIONED_TOPIC_PATH, namespaceName.toString());
+
+            return existsAsync(globalPartitionedPath).thenAccept(exist -> {

Review comment:
       If you use `thenCompose` here, we still need to deal with the return value, their logic is the same




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] gaoran10 removed a comment on pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
gaoran10 removed a comment on pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#issuecomment-964976148


   Hi, @wolfstudy, could you add a unit test for this PR?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r746340573



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java
##########
@@ -122,6 +123,16 @@ public static boolean pathIsFromNamespace(String path) {
                 && path.substring(BASE_POLICIES_PATH.length() + 1).contains("/");
     }
 
+    // clear resource of `/namespace/{namespaceName}` for zk-node
+    public void clearNamespace(NamespaceName ns) throws MetadataStoreException {

Review comment:
       ```suggestion
       public void deleteNamespace(NamespaceName ns) throws MetadataStoreException {
   ```

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java
##########
@@ -269,6 +270,23 @@ protected void internalDeleteTenant(AsyncResponse asyncResponse, String tenant)
                         asyncResponse.resume(new RestException(ex));
                         return null;
                     });
+
+                    // After clearing TenantResources, we need to clear the information of zk-node at the same time
+                    try {
+                        pulsar().getPulsarResources().getTopicResources().clearTenantPersistence(tenant).get();

Review comment:
       Avoid calling a block method in a async callback.

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java
##########
@@ -269,6 +270,23 @@ protected void internalDeleteTenant(AsyncResponse asyncResponse, String tenant)
                         asyncResponse.resume(new RestException(ex));
                         return null;
                     });
+
+                    // After clearing TenantResources, we need to clear the information of zk-node at the same time
+                    try {
+                        pulsar().getPulsarResources().getTopicResources().clearTenantPersistence(tenant).get();

Review comment:
       Please check all.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java
##########
@@ -122,6 +123,16 @@ public static boolean pathIsFromNamespace(String path) {
                 && path.substring(BASE_POLICIES_PATH.length() + 1).contains("/");
     }
 
+    // clear resource of `/namespace/{namespaceName}` for zk-node
+    public void clearNamespace(NamespaceName ns) throws MetadataStoreException {
+        delete(joinPath(OWNER_INFO_ROOT, ns.toString()));
+    }
+
+    // clear resource of `/namespace/{tenant}` for zk-node
+    public void clearTenant(String tenant) throws MetadataStoreException {

Review comment:
       ```suggestion
       public void deleteTenant(String tenant) throws MetadataStoreException {
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r748685665



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java
##########
@@ -215,6 +258,17 @@ public boolean partitionedTopicExists(TopicName tn) throws MetadataStoreExceptio
                     tn.getEncodedLocalName()));
         }
 
+        public CompletableFuture<Void> clearPartitionedTopicMetadataAsync(NamespaceName namespaceName) {
+            final String globalPartitionedPath = joinPath(PARTITIONED_TOPIC_PATH, namespaceName.toString());
+
+            return existsAsync(globalPartitionedPath).thenAccept(exist -> {

Review comment:
       In here, we don’t need a return value, so using `thenAccpet` is OK.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r748361499



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java
##########
@@ -181,19 +182,46 @@ protected static String joinPath(String... parts) {
         return sb.toString();
     }
 
-
-
     protected static void deleteRecursive(BaseResources resources, final String pathRoot) throws MetadataStoreException {
         PathUtils.validatePath(pathRoot);
         List<String> tree = listSubTreeBFS(resources, pathRoot);
         log.debug("Deleting {} with size {}", tree, tree.size());
-        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
             resources.delete(tree.get(i));
         }
     }
 
+    protected static CompletableFuture<Void> deleteRecursiveAsync(BaseResources resources, final String pathRoot) {
+        PathUtils.validatePath(pathRoot);
+        List<String> tree = null;
+        try {
+            tree = listSubTreeBFS(resources, pathRoot);
+        } catch (MetadataStoreException e) {
+            // no-op
+        }
+
+        if (tree != null) {
+            log.debug("Deleting {} with size {}", tree, tree.size());
+
+            final List<CompletableFuture<Void>> futures = new ArrayList<>();
+            for (int i = tree.size() - 1; i >= 0; --i) {
+                // Delete the leaves first and eventually get rid of the root
+                futures.add(resources.deleteAsync(tree.get(i)));
+            }
+
+            FutureUtil.waitForAll(futures).handle((result, exception) -> {
+                if (exception != null) {
+                    log.error("Failed to remove partitioned topics", exception);
+                    return null;
+                }
+                return null;
+            });
+        }
+
+        return null;

Review comment:
       We can't return null here, should be a completed future.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r747641668



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java
##########
@@ -194,6 +197,38 @@ protected static void deleteRecursive(BaseResources resources, final String path
         }
     }
 
+    protected static CompletableFuture<Void> deleteRecursiveAsync(BaseResources resources, final String pathRoot) {
+        PathUtils.validatePath(pathRoot);
+        List<String> tree = null;
+        try {
+            tree = listSubTreeBFS(resources, pathRoot);
+        } catch (MetadataStoreException e) {
+

Review comment:
       Yes, we can ignore this exception.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r749947958



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java
##########
@@ -215,12 +258,27 @@ public boolean partitionedTopicExists(TopicName tn) throws MetadataStoreExceptio
                     tn.getEncodedLocalName()));
         }
 
-        public void clearPartitionedTopicMetadata(NamespaceName namespaceName) throws MetadataStoreException {
+        public CompletableFuture<Void> clearPartitionedTopicMetadataAsync(NamespaceName namespaceName) {
             final String globalPartitionedPath = joinPath(PARTITIONED_TOPIC_PATH, namespaceName.toString());
-            // check whether partitioned topics metadata node exist
-            if (exists(globalPartitionedPath)) {
-                deleteRecursive(this, globalPartitionedPath);
-            }
+
+            CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+
+            deleteRecursiveAsync(this, globalPartitionedPath)
+                    .thenAccept(ignore -> {
+                        log.info("Clear partitioned topic metadata [{}] success.", namespaceName);
+                        completableFuture.complete(null);
+                    }).exceptionally(ex -> {
+                if (ex.getCause().getCause() instanceof KeeperException) {

Review comment:
       We should not skip all the KeeperException? on the no node exist exception?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy commented on a change in pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#discussion_r747941011



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
##########
@@ -484,42 +474,39 @@ protected void internalDeleteNamespaceForcefully(AsyncResponse asyncResponse, bo
                     asyncResponse.resume(new RestException((PulsarAdminException) exception.getCause()));
                     return null;
                 } else {
-                    log.error("[{}] Failed to remove owned namespace {}", clientAppId(), namespaceName, exception);
+                    log.error("[{}] Failed to remove forcefully owned namespace {}",
+                            clientAppId(), namespaceName, exception);
                     asyncResponse.resume(new RestException(exception.getCause()));
                     return null;
                 }
             }
 
-            try {
-                // remove partitioned topics znode
-                pulsar().getPulsarResources().getNamespaceResources().getPartitionedTopicResources()
-                        .clearPartitionedTopicMetadata(namespaceName);
-
-                try {
-                    pulsar().getPulsarResources().getTopicResources().clearDomainPersistence(namespaceName).get();
-                    pulsar().getPulsarResources().getTopicResources().clearNamespacePersistence(namespaceName).get();
-                } catch (ExecutionException | InterruptedException e) {
-                    // warn level log here since this failure has no side effect besides left a un-used metadata
-                    // and also will not affect the re-creation of namespace
-                    log.warn("[{}] Failed to remove managed-ledger for {}", clientAppId(), namespaceName, e);
-                }
-
-                // we have successfully removed all the ownership for the namespace, the policies znode can be deleted
-                // now
-                namespaceResources().deletePolicies(namespaceName);
-
-                try {
-                    getLocalPolicies().deleteLocalPolicies(namespaceName);
-                } catch (NotFoundException nne) {
-                    // If the z-node with the modified information is not there anymore, we're already good
-                }
-            } catch (Exception e) {
-                log.error("[{}] Failed to remove owned namespace {} from ZK", clientAppId(), namespaceName, e);
-                asyncResponse.resume(new RestException(e));
-                return null;
-            }
+            // clear resource of `/namespace/{namespaceName}` for zk-node
+            namespaceResources().deleteNamespaceAsync(namespaceName)
+                    .thenCompose(ignore -> namespaceResources().getPartitionedTopicResources()
+                            .clearPartitionedTopicMetadataAsync(namespaceName))
+                    // clear resource for manager-ledger z-node
+                    .thenCompose(ignore -> pulsar().getPulsarResources().getTopicResources()
+                            .clearDomainPersistence(namespaceName))
+                    .thenCompose(ignore -> pulsar().getPulsarResources().getTopicResources()
+                            .clearNamespacePersistence(namespaceName))
+                    // we have successfully removed all the ownership for the namespace, the policies
+                    // z-node can be deleted now
+                    .thenCompose(ignore -> namespaceResources().deletePoliciesAsync(namespaceName))
+                    // clear z-node of local policies
+                    .thenCompose(ignore -> getLocalPolicies().deleteLocalPoliciesAsync(namespaceName))
+                    .whenComplete((ignore, ex) -> {
+                        if (ex != null) {
+                            log.warn("[{}] Failed to force remove namespace or managed-ledger for {}",
+                                    clientAppId(), namespaceName, ex);
+                            asyncResponse.resume(new RestException(ex));
+                        } else {
+                            log.info("[{}] Remove forcefully namespace or managed-ledger successfully {}",
+                                    clientAppId(), namespaceName);
+                            asyncResponse.resume(Response.noContent().build());
+                        }
+                    });
 

Review comment:
       Good idea, will process it




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wolfstudy merged pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
wolfstudy merged pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] github-actions[bot] commented on pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#issuecomment-964843963


   @wolfstudy:Thanks for your contribution. For this PR, do we need to update docs?
   (The [PR template contains info about doc](https://github.com/apache/pulsar/blob/master/.github/PULL_REQUEST_TEMPLATE.md#documentation), which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on pull request #12711: Fix znode leakage caused by deleting tenant

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on pull request #12711:
URL: https://github.com/apache/pulsar/pull/12711#issuecomment-964887709


   @wolfstudy Please help add a test to avoid the regression.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org