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 11:38:41 UTC

[GitHub] [pulsar] codelipenghui opened a new pull request #10383: Fix schema ledger deletion when deleting topic with delete schema.

codelipenghui opened a new pull request #10383:
URL: https://github.com/apache/pulsar/pull/10383


   Related to #8167
   
   ### Motivation
   
   Fix schema ledger deletion when deleting topic with delete schema.
   
   ### Modifications
   
   Delete ledgers according to the schema index.
   
   ### Verifying this change
   
   New unit tests added.
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API: (no)
     - The schema: (no)
     - The default values of configurations: (no)
     - The wire protocol: (no)
     - The rest endpoints: (no)
     - The admin cli options: (no)
     - Anything that affects deployment: (no)
   
   ### Documentation
   
     - Does this pull request introduce a new feature? (no)
   


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



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

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
##########
@@ -396,33 +391,41 @@ 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);
+                            } catch (InterruptedException | KeeperException thr) {
+                                future.completeExceptionally(thr);

Review comment:
       Nice catch!

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
##########
@@ -396,33 +391,41 @@ 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) -> {

Review comment:
       No, one ledger delete failed, we only print the warn log instead failed the whole delete schema operation.




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



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

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
##########
@@ -396,33 +391,41 @@ 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) -> {

Review comment:
       If one ledger delete failed, how we clean it again?

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
##########
@@ -396,33 +391,41 @@ 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) -> {

Review comment:
       If one ledger delete failed, does it need to clean it again? If delete zk path of the schema, we couldn't find its ledgers again.




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



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

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



##########
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:
       Fixed, PTAL.




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



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

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
##########
@@ -396,33 +391,41 @@ 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) -> {

Review comment:
       `whenComplete((v, e)` need to handle the exception?

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
##########
@@ -396,33 +391,41 @@ 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);
+                            } catch (InterruptedException | KeeperException thr) {
+                                future.completeExceptionally(thr);

Review comment:
       should return after `future.completeExceptionally(thr)`




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



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

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



##########
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:
       @merlimat Fixed, PTAL.




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



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

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


   @BewareMyPower Please also help review 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.

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



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

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


   @merlimat please take your final look,
   we would like to pick this to 2.7.2, so it is blocking the release


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



[GitHub] [pulsar] merlimat merged pull request #10383: Fix schema ledger deletion when deleting topic with delete schema.

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


   


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



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

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
##########
@@ -396,33 +391,41 @@ 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) -> {

Review comment:
       If one ledger delete failed, does it need to be cleaned again? If delete zk path of the schema, we couldn't find its ledgers again.




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



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

Posted by GitBox <gi...@apache.org>.
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