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/04/26 19:04:24 UTC

[GitHub] [pulsar] merlimat commented on a change in pull request #10383: Fix schema ledger deletion when deleting topic with delete schema.

merlimat commented on a change in pull request #10383:
URL: https://github.com/apache/pulsar/pull/10383#discussion_r620571645



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
##########
@@ -396,33 +391,42 @@ public void close() throws Exception {
             } else {
                 // The version is only for the compatibility of the current interface
                 final long version = -1;
-                final List<Long> ledgerIds = schemaLedgers.get(schemaId);
-                if (ledgerIds != null) {
-                    CompletableFuture<Long> future = new CompletableFuture<>();
-                    final AtomicInteger numOfLedgerIds = new AtomicInteger(ledgerIds.size());
-                    for (long ledgerId : ledgerIds) {
-                        bookKeeper.asyncDeleteLedger(ledgerId, (int rc, Object cnx) -> {
-                            if (rc != BKException.Code.OK) {
-                                // It's not a serious error, we didn't need call future.completeExceptionally()
-                                log.warn("Failed to delete ledger {} of {}: {}", ledgerId, schemaId, rc);
-                            }
-                            if (numOfLedgerIds.decrementAndGet() == 0) {
-                                try {
-                                    ZkUtils.deleteFullPathOptimistic(zooKeeper, getSchemaPath(schemaId), -1);
-                                } catch (InterruptedException | KeeperException e) {
-                                    future.completeExceptionally(e);
+                CompletableFuture<Long> future = new CompletableFuture<>();
+                getLocator(schemaId).whenComplete((locator, ex) -> {
+                    if (ex != null) {
+                        future.completeExceptionally(ex);
+                    } else {
+                        if (!locator.isPresent()) {
+                            future.complete(null);
+                            return;
+                        }
+                        List<SchemaStorageFormat.IndexEntry> indexEntryList = locator.get().locator.getIndexList();
+                        List<CompletableFuture<Void>> deleteFutures = new ArrayList<>(indexEntryList.size());
+                        indexEntryList.forEach(indexEntry -> {
+                            final long ledgerId = indexEntry.getPosition().getLedgerId();
+                            CompletableFuture<Void> deleteFuture = new CompletableFuture<>();
+                            deleteFutures.add(deleteFuture);
+                            bookKeeper.asyncDeleteLedger(ledgerId, (int rc, Object cnx) -> {
+                                if (rc != BKException.Code.OK) {
+                                    // It's not a serious error, we didn't need call future.completeExceptionally()
+                                    log.warn("Failed to delete ledger {} of {}: {}", ledgerId, schemaId, rc);
                                 }
-                                clearLocatorCache(getSchemaPath(schemaId));
-                                future.complete(version);
+                                deleteFuture.complete(null);
+                            }, null);
+                        });
+                        FutureUtil.waitForAll(deleteFutures).whenComplete((v, e) -> {
+                            try {
+                                ZkUtils.deleteFullPathOptimistic(zooKeeper, getSchemaPath(schemaId), -1);

Review comment:
       This call can be dangerous (I think it was already the case in the existing code). We're making a blocking call from the callback of an async call. It can lead to deadlock the ZK thread. We should never mix sync/async calls, rather we should convert into using `ZkUtils.asyncDeleteFullPathOptimistic()`




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