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 2022/06/02 20:43:07 UTC

[GitHub] [pulsar] dlg99 opened a new pull request, #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

dlg99 opened a new pull request, #15914:
URL: https://github.com/apache/pulsar/pull/15914

   
   Fixes #9962 
   
   ### Motivation
   
   Offloaded ledgers can be orphaned on topic deletion. 
   
   ### Modifications
   
   Topic to be truncated before deletion to delete offloaded ledgers properly and fail if truncation fails.
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   This change added integration tests
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
   Nothing changed in the options but admin CLI will implicitly run truncate before topic delete. 
   
     - Dependencies (does it add or upgrade a dependency): (yes / no)
     - The public API: (yes / no)
     - The schema: (yes / no / don't know)
     - The default values of configurations: (yes / no)
     - The wire protocol: (yes / no)
     - The rest endpoints: (yes / no)
     - The admin cli options: (yes / no)
     - Anything that affects deployment: (yes / no / don't know)
   
   ### Documentation
   
   Check the box below or label this PR directly.
   
   Need to update docs? 
   
   - [ ] `doc-required` 
   (Your PR needs to update docs and you will update later)
     
   - [x] `doc-not-needed` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)


-- 
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] Jason918 commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r973532353


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java:
##########
@@ -1072,28 +1074,31 @@ public CompletableFuture<Void> deleteTopic(String topic, boolean forceDelete) {
         }
 
         CompletableFuture<Void> future = new CompletableFuture<>();
+        log.info("Topic {} could not load, try to delete from metadata", topic);
 
         CompletableFuture<Void> deleteTopicAuthenticationFuture = new CompletableFuture<>();
         deleteTopicAuthenticationWithRetry(topic, deleteTopicAuthenticationFuture, 5);
+
         deleteTopicAuthenticationFuture.whenComplete((v, ex) -> {
             if (ex != null) {
                 future.completeExceptionally(ex);
                 return;
             }
-            managedLedgerFactory.asyncDelete(tn.getPersistenceNamingEncoding(), new DeleteLedgerCallback() {
-                @Override
-                public void deleteLedgerComplete(Object ctx) {
-                    future.complete(null);
-                }
+            CompletableFuture<ManagedLedgerConfig> mlConfigFuture =  getManagedLedgerConfig(topicName);

Review Comment:
   ```suggestion
               CompletableFuture<ManagedLedgerConfig> mlConfigFuture = getManagedLedgerConfig(topicName);
   ```



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java:
##########
@@ -154,6 +164,18 @@ void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosition, M
      */
     void asyncDelete(String name, DeleteLedgerCallback callback, Object ctx);
 
+    /**
+     * Delete a managed ledger. If it's not open, it's metadata will get regardless deleted.
+     *
+     * @param name
+     * @throws InterruptedException
+     * @throws ManagedLedgerException
+     */
+    default void asyncDelete(String name, CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
+                             DeleteLedgerCallback callback, Object ctx) {
+        asyncDelete(name, callback, ctx);

Review Comment:
   NIT: No need to add this default implementation as we already overrode this.



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -873,23 +891,101 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
         }, ctx);
     }
 
+    private CompletableFuture<Void> cleanupOffloaded(long ledgerId, UUID uuid, ManagedLedgerConfig mlConfig,
+                                     Map<String, String> offloadDriverMetadata, String cleanupReason, String name) {
+        log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.",
+                name, ledgerId, uuid.toString(), cleanupReason);
+        Map<String, String> metadataMap = new HashMap();
+        metadataMap.putAll(offloadDriverMetadata);
+        metadataMap.put("ManagedLedgerName", name);
+
+        return Retries.run(Backoff.exponentialJittered(TimeUnit.SECONDS.toMillis(1),
+                        TimeUnit.SECONDS.toHours(1)).limit(10),
+                Retries.NonFatalPredicate,
+                () -> mlConfig.getLedgerOffloader().deleteOffloaded(ledgerId, uuid, metadataMap),
+                scheduledExecutor, name).whenComplete((ignored, exception) -> {
+            if (exception != null) {
+                log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})",
+                        name, ledgerId, cleanupReason, exception);
+            }
+        });
+    }
+
     private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, ManagedLedgerInfo info,
-            DeleteLedgerCallback callback, Object ctx) {
+                                         CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
+                                         DeleteLedgerCallback callback, Object ctx) {
+        final CompletableFuture<Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo>>
+                ledgerInfosFuture = new CompletableFuture<>();
+        store.getManagedLedgerInfo(managedLedgerName, false, null,

Review Comment:
   Why do we read `ManagedLedgerInfo` from store instead of just use `info` in the parameter list? 



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java:
##########
@@ -1072,28 +1074,31 @@ public CompletableFuture<Void> deleteTopic(String topic, boolean forceDelete) {
         }
 
         CompletableFuture<Void> future = new CompletableFuture<>();
+        log.info("Topic {} could not load, try to delete from metadata", topic);

Review Comment:
   Duplicate with debug log in L1067?



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -873,23 +891,101 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
         }, ctx);
     }
 
+    private CompletableFuture<Void> cleanupOffloaded(long ledgerId, UUID uuid, ManagedLedgerConfig mlConfig,

Review Comment:
   Can reuse the code with `ManagedLedgerImpl#cleanupOffloaded`, it's just the same.
   Maybe put the codes in `OffloaderUtils`?



-- 
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] Jason918 commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r983270598


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -873,23 +891,101 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
         }, ctx);
     }
 
+    private CompletableFuture<Void> cleanupOffloaded(long ledgerId, UUID uuid, ManagedLedgerConfig mlConfig,
+                                     Map<String, String> offloadDriverMetadata, String cleanupReason, String name) {
+        log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.",
+                name, ledgerId, uuid.toString(), cleanupReason);
+        Map<String, String> metadataMap = new HashMap();
+        metadataMap.putAll(offloadDriverMetadata);
+        metadataMap.put("ManagedLedgerName", name);
+
+        return Retries.run(Backoff.exponentialJittered(TimeUnit.SECONDS.toMillis(1),
+                        TimeUnit.SECONDS.toHours(1)).limit(10),
+                Retries.NonFatalPredicate,
+                () -> mlConfig.getLedgerOffloader().deleteOffloaded(ledgerId, uuid, metadataMap),
+                scheduledExecutor, name).whenComplete((ignored, exception) -> {
+            if (exception != null) {
+                log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})",
+                        name, ledgerId, cleanupReason, exception);
+            }
+        });
+    }
+
     private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, ManagedLedgerInfo info,
-            DeleteLedgerCallback callback, Object ctx) {
+                                         CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
+                                         DeleteLedgerCallback callback, Object ctx) {
+        final CompletableFuture<Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo>>
+                ledgerInfosFuture = new CompletableFuture<>();
+        store.getManagedLedgerInfo(managedLedgerName, false, null,

Review Comment:
   > Unfortunately ManagedLedgerInfo is part of of the public REST API
   
   Sorry, I missed this. 
   
   
   > I prefer to keep the patch in this form. And if we want to change ManagedLedgerInfo we can do it in a follow up work.
   > 
   > As this patch is fixing some kind of "bad problem" (because we are not deleting data that should have been deleted, that has some legal impact in some countries), this patch should be cherry-picked to active branches. I won't add API changes in a patch that will be ported
   
   Make sense to me, this patch LGTM



-- 
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] hangc0276 commented on pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#issuecomment-1146942281

   Even though we should delete the offload data on topic deletion, it has changed the default behavior. We'd better have a proposal to discuss it.
   
   Another one is that it can't prevent orphan ledgers because they offloaded data deletion is async.


-- 
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] horizonzy commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
horizonzy commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r906745657


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -2687,6 +2687,22 @@ public void deleteLedgerFailed(ManagedLedgerException e, Object ctx) {
 
     @Override
     public void asyncDelete(final DeleteLedgerCallback callback, final Object ctx) {
+        // Truncate to ensure the offloaded data is not orphaned.
+        // Also ensures the BK ledgers are deleted and not just scheduled for deletion
+        CompletableFuture<Void> truncateFuture = asyncTruncate();

Review Comment:
   If we use `asyncTruncate` to trigger delete storage data.
   Maybe we should mofidy the code:
   https://github.com/apache/pulsar/blob/ce24db11adaf60d9cfb302ce39e76cdc9e43bd2d/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L2430-L2434
   
   When the meta data service is not available, should complete with 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] horizonzy commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
horizonzy commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r894299875


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -1196,35 +1196,48 @@ private CompletableFuture<Void> delete(boolean failIfHasSubscriptions,
                                     unfenceTopicToResume();
                                     deleteFuture.completeExceptionally(e);
                                 } else {
-                                    ledger.asyncDelete(new AsyncCallbacks.DeleteLedgerCallback() {
-                                        @Override
-                                        public void deleteLedgerComplete(Object ctx) {
-                                            brokerService.removeTopicFromCache(topic);
-
-                                            dispatchRateLimiter.ifPresent(DispatchRateLimiter::close);
-
-                                            subscribeRateLimiter.ifPresent(SubscribeRateLimiter::close);
-
-                                            unregisterTopicPolicyListener();
-
-                                            log.info("[{}] Topic deleted", topic);
-                                            deleteFuture.complete(null);
-                                        }
+                                    // Truncate to ensure the offloaded data is not orphaned.
+                                    // Also ensures the BK ledgers are deleted and not just scheduled for deletion
+                                    CompletableFuture<Void> truncateFuture = ledger.asyncTruncate();
+                                    truncateFuture.whenComplete((ignore, exc) -> {

Review Comment:
   How about that move this logic to ledger.asyncDelete(). It can cover more situation.



-- 
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] eolivelli merged pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
eolivelli merged PR #15914:
URL: https://github.com/apache/pulsar/pull/15914


-- 
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] horizonzy commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
horizonzy commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r894299875


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -1196,35 +1196,48 @@ private CompletableFuture<Void> delete(boolean failIfHasSubscriptions,
                                     unfenceTopicToResume();
                                     deleteFuture.completeExceptionally(e);
                                 } else {
-                                    ledger.asyncDelete(new AsyncCallbacks.DeleteLedgerCallback() {
-                                        @Override
-                                        public void deleteLedgerComplete(Object ctx) {
-                                            brokerService.removeTopicFromCache(topic);
-
-                                            dispatchRateLimiter.ifPresent(DispatchRateLimiter::close);
-
-                                            subscribeRateLimiter.ifPresent(SubscribeRateLimiter::close);
-
-                                            unregisterTopicPolicyListener();
-
-                                            log.info("[{}] Topic deleted", topic);
-                                            deleteFuture.complete(null);
-                                        }
+                                    // Truncate to ensure the offloaded data is not orphaned.
+                                    // Also ensures the BK ledgers are deleted and not just scheduled for deletion
+                                    CompletableFuture<Void> truncateFuture = ledger.asyncTruncate();
+                                    truncateFuture.whenComplete((ignore, exc) -> {

Review Comment:
   How about move this logic to ledger.asyncDelete(). It can cover more situation.



-- 
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] hangc0276 commented on pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#issuecomment-1146942741

   We need more eyes. @merlimat @codelipenghui @315157973 @Jason918 


-- 
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] dlg99 commented on pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
dlg99 commented on PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#issuecomment-1249686046

   @eolivelli @hangc0276 please take another look.
   I addressed the case when the topic is unloaded.


-- 
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] Jason918 commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r979182179


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -873,23 +891,101 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
         }, ctx);
     }
 
+    private CompletableFuture<Void> cleanupOffloaded(long ledgerId, UUID uuid, ManagedLedgerConfig mlConfig,
+                                     Map<String, String> offloadDriverMetadata, String cleanupReason, String name) {
+        log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.",
+                name, ledgerId, uuid.toString(), cleanupReason);
+        Map<String, String> metadataMap = new HashMap();
+        metadataMap.putAll(offloadDriverMetadata);
+        metadataMap.put("ManagedLedgerName", name);
+
+        return Retries.run(Backoff.exponentialJittered(TimeUnit.SECONDS.toMillis(1),
+                        TimeUnit.SECONDS.toHours(1)).limit(10),
+                Retries.NonFatalPredicate,
+                () -> mlConfig.getLedgerOffloader().deleteOffloaded(ledgerId, uuid, metadataMap),
+                scheduledExecutor, name).whenComplete((ignored, exception) -> {
+            if (exception != null) {
+                log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})",
+                        name, ledgerId, cleanupReason, exception);
+            }
+        });
+    }
+
     private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, ManagedLedgerInfo info,
-            DeleteLedgerCallback callback, Object ctx) {
+                                         CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
+                                         DeleteLedgerCallback callback, Object ctx) {
+        final CompletableFuture<Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo>>
+                ledgerInfosFuture = new CompletableFuture<>();
+        store.getManagedLedgerInfo(managedLedgerName, false, null,

Review Comment:
   This still confuse me. All data in `ManagedLedgerInfo` comes directly from `MLDataFormats.ManagedLedgerInfo.LedgerInfo`. I think it's better to just sync all the info to `ManagedLedgerInfo`.



-- 
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] congbobo184 commented on pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#issuecomment-1314658172

   could you please cherry-pick this PR to branch-2.9? 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] horizonzy commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
horizonzy commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r894299875


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -1196,35 +1196,48 @@ private CompletableFuture<Void> delete(boolean failIfHasSubscriptions,
                                     unfenceTopicToResume();
                                     deleteFuture.completeExceptionally(e);
                                 } else {
-                                    ledger.asyncDelete(new AsyncCallbacks.DeleteLedgerCallback() {
-                                        @Override
-                                        public void deleteLedgerComplete(Object ctx) {
-                                            brokerService.removeTopicFromCache(topic);
-
-                                            dispatchRateLimiter.ifPresent(DispatchRateLimiter::close);
-
-                                            subscribeRateLimiter.ifPresent(SubscribeRateLimiter::close);
-
-                                            unregisterTopicPolicyListener();
-
-                                            log.info("[{}] Topic deleted", topic);
-                                            deleteFuture.complete(null);
-                                        }
+                                    // Truncate to ensure the offloaded data is not orphaned.
+                                    // Also ensures the BK ledgers are deleted and not just scheduled for deletion
+                                    CompletableFuture<Void> truncateFuture = ledger.asyncTruncate();
+                                    truncateFuture.whenComplete((ignore, exc) -> {

Review Comment:
   How about moving this logic to ledger.asyncDelete(). It can cover more situation.



-- 
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] eolivelli commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
eolivelli commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r983183760


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -873,23 +891,101 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
         }, ctx);
     }
 
+    private CompletableFuture<Void> cleanupOffloaded(long ledgerId, UUID uuid, ManagedLedgerConfig mlConfig,
+                                     Map<String, String> offloadDriverMetadata, String cleanupReason, String name) {
+        log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.",
+                name, ledgerId, uuid.toString(), cleanupReason);
+        Map<String, String> metadataMap = new HashMap();
+        metadataMap.putAll(offloadDriverMetadata);
+        metadataMap.put("ManagedLedgerName", name);
+
+        return Retries.run(Backoff.exponentialJittered(TimeUnit.SECONDS.toMillis(1),
+                        TimeUnit.SECONDS.toHours(1)).limit(10),
+                Retries.NonFatalPredicate,
+                () -> mlConfig.getLedgerOffloader().deleteOffloaded(ledgerId, uuid, metadataMap),
+                scheduledExecutor, name).whenComplete((ignored, exception) -> {
+            if (exception != null) {
+                log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})",
+                        name, ledgerId, cleanupReason, exception);
+            }
+        });
+    }
+
     private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, ManagedLedgerInfo info,
-            DeleteLedgerCallback callback, Object ctx) {
+                                         CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
+                                         DeleteLedgerCallback callback, Object ctx) {
+        final CompletableFuture<Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo>>
+                ledgerInfosFuture = new CompletableFuture<>();
+        store.getManagedLedgerInfo(managedLedgerName, false, null,

Review Comment:
   Unfortunately ManagedLedgerInfo is part of of the public REST API
   
   for instance here we return it (JSON encoded) to the client
   https://github.com/apache/pulsar/blob/957a16b341197d6734f170a5cdcdd0500133c771/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java#L1262
   
   We should not add everything to it.
   
   We have this bad problem in Pulsar that we aren't always aware of what is leaking to the public APIs.
   
   I prefer to keep the patch in this form.
   And if we want to change ManagedLedgerInfo we can do it in a follow up work.
   
   As this patch is fixing some kind of "bad problem" (because we are not deleting data that should have been deleted, that has some legal impact in some countries), this patch should be cherry-picked to active branches.
   I won't add API changes in a patch that will be ported
   
   



-- 
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] congbobo184 commented on pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#issuecomment-1318513531

   @dlg99 hi, I move this PR to `release/2.9.5`, if you have any questions, please ping me. 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] dlg99 commented on pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
dlg99 commented on PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#issuecomment-1254438383

   @Jason918 I addressed your comments


-- 
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] Jason918 commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r983270598


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -873,23 +891,101 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
         }, ctx);
     }
 
+    private CompletableFuture<Void> cleanupOffloaded(long ledgerId, UUID uuid, ManagedLedgerConfig mlConfig,
+                                     Map<String, String> offloadDriverMetadata, String cleanupReason, String name) {
+        log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.",
+                name, ledgerId, uuid.toString(), cleanupReason);
+        Map<String, String> metadataMap = new HashMap();
+        metadataMap.putAll(offloadDriverMetadata);
+        metadataMap.put("ManagedLedgerName", name);
+
+        return Retries.run(Backoff.exponentialJittered(TimeUnit.SECONDS.toMillis(1),
+                        TimeUnit.SECONDS.toHours(1)).limit(10),
+                Retries.NonFatalPredicate,
+                () -> mlConfig.getLedgerOffloader().deleteOffloaded(ledgerId, uuid, metadataMap),
+                scheduledExecutor, name).whenComplete((ignored, exception) -> {
+            if (exception != null) {
+                log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})",
+                        name, ledgerId, cleanupReason, exception);
+            }
+        });
+    }
+
     private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, ManagedLedgerInfo info,
-            DeleteLedgerCallback callback, Object ctx) {
+                                         CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
+                                         DeleteLedgerCallback callback, Object ctx) {
+        final CompletableFuture<Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo>>
+                ledgerInfosFuture = new CompletableFuture<>();
+        store.getManagedLedgerInfo(managedLedgerName, false, null,

Review Comment:
   > Unfortunately ManagedLedgerInfo is part of of the public REST API
   
   Sorry, I missed this. 
   
   
   > I prefer to keep the patch in this form. And if we want to change ManagedLedgerInfo we can do it in a follow up work.
   > 
   > As this patch is fixing some kind of "bad problem" (because we are not deleting data that should have been deleted, that has some legal impact in some countries), this patch should be cherry-picked to active branches. I won't add API changes in a patch that will be ported
   
   Make sense to me, this patch is good to go.



-- 
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] eolivelli commented on pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

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

   tests are now failing due to https://github.com/apache/pulsar/pull/17736
   
   because "trim" cannot happen if a ManagedLedger is "fenced", and we set "fenced" in "delete".
   so there is some code path in which we "delete" and then try to "trim".
   we should do the opposite...trim and then delete.
   
   


-- 
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] dlg99 commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
dlg99 commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r976990811


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -873,23 +891,101 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
         }, ctx);
     }
 
+    private CompletableFuture<Void> cleanupOffloaded(long ledgerId, UUID uuid, ManagedLedgerConfig mlConfig,
+                                     Map<String, String> offloadDriverMetadata, String cleanupReason, String name) {
+        log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.",
+                name, ledgerId, uuid.toString(), cleanupReason);
+        Map<String, String> metadataMap = new HashMap();
+        metadataMap.putAll(offloadDriverMetadata);
+        metadataMap.put("ManagedLedgerName", name);
+
+        return Retries.run(Backoff.exponentialJittered(TimeUnit.SECONDS.toMillis(1),
+                        TimeUnit.SECONDS.toHours(1)).limit(10),
+                Retries.NonFatalPredicate,
+                () -> mlConfig.getLedgerOffloader().deleteOffloaded(ledgerId, uuid, metadataMap),
+                scheduledExecutor, name).whenComplete((ignored, exception) -> {
+            if (exception != null) {
+                log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})",
+                        name, ledgerId, cleanupReason, exception);
+            }
+        });
+    }
+
     private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, ManagedLedgerInfo info,
-            DeleteLedgerCallback callback, Object ctx) {
+                                         CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
+                                         DeleteLedgerCallback callback, Object ctx) {
+        final CompletableFuture<Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo>>
+                ledgerInfosFuture = new CompletableFuture<>();
+        store.getManagedLedgerInfo(managedLedgerName, false, null,

Review Comment:
   `info` is `org.apache.bookkeeper.mledger.ManagedLedgerInfo`, store returns `MLDataFormats.ManagedLedgerInfo.LedgerInfo` which has some additional info and used in OffloadUtils
   



-- 
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] dlg99 commented on pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
dlg99 commented on PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#issuecomment-1147688612

   @hangc0276 
   > Even though we should delete the offload data on topic deletion, it has changed the default behavior. We'd better have a proposal to discuss it.
   
   I disagree that undocumented silently orphaned data (without tools/automated processes to detect it and clean up) is a "default behavior" one ever expected or wanted. It looks like a bug that just happened.
   
   > Another one is that it can't prevent orphan ledgers because they offloaded data deletion is async.
   
   In case of truncate internalTrimLedgers runs with `isTruncate == true` actual future that completes when trimming is done, and the topic deletion will follow that. 
   In case of deletion we have `isTruncate == false` and  internalTrimLedgers runs with Futures.NULL_PROMISE.
    
   In case I missed some place where internalTrimLedgers completes the promise before the data deletion is done i'll fix that as long as we agree on overall approach.


-- 
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 #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

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

   The pr had no activity for 30 days, mark with Stale label.


-- 
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] dlg99 commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
dlg99 commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r903186123


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -1196,35 +1196,48 @@ private CompletableFuture<Void> delete(boolean failIfHasSubscriptions,
                                     unfenceTopicToResume();
                                     deleteFuture.completeExceptionally(e);
                                 } else {
-                                    ledger.asyncDelete(new AsyncCallbacks.DeleteLedgerCallback() {
-                                        @Override
-                                        public void deleteLedgerComplete(Object ctx) {
-                                            brokerService.removeTopicFromCache(topic);
-
-                                            dispatchRateLimiter.ifPresent(DispatchRateLimiter::close);
-
-                                            subscribeRateLimiter.ifPresent(SubscribeRateLimiter::close);
-
-                                            unregisterTopicPolicyListener();
-
-                                            log.info("[{}] Topic deleted", topic);
-                                            deleteFuture.complete(null);
-                                        }
+                                    // Truncate to ensure the offloaded data is not orphaned.
+                                    // Also ensures the BK ledgers are deleted and not just scheduled for deletion
+                                    CompletableFuture<Void> truncateFuture = ledger.asyncTruncate();
+                                    truncateFuture.whenComplete((ignore, exc) -> {

Review Comment:
   @horizonzy moved it into ManagedLedgerImpl



-- 
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] Jason918 commented on a diff in pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#discussion_r979182179


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -873,23 +891,101 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
         }, ctx);
     }
 
+    private CompletableFuture<Void> cleanupOffloaded(long ledgerId, UUID uuid, ManagedLedgerConfig mlConfig,
+                                     Map<String, String> offloadDriverMetadata, String cleanupReason, String name) {
+        log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.",
+                name, ledgerId, uuid.toString(), cleanupReason);
+        Map<String, String> metadataMap = new HashMap();
+        metadataMap.putAll(offloadDriverMetadata);
+        metadataMap.put("ManagedLedgerName", name);
+
+        return Retries.run(Backoff.exponentialJittered(TimeUnit.SECONDS.toMillis(1),
+                        TimeUnit.SECONDS.toHours(1)).limit(10),
+                Retries.NonFatalPredicate,
+                () -> mlConfig.getLedgerOffloader().deleteOffloaded(ledgerId, uuid, metadataMap),
+                scheduledExecutor, name).whenComplete((ignored, exception) -> {
+            if (exception != null) {
+                log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})",
+                        name, ledgerId, cleanupReason, exception);
+            }
+        });
+    }
+
     private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, ManagedLedgerInfo info,
-            DeleteLedgerCallback callback, Object ctx) {
+                                         CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
+                                         DeleteLedgerCallback callback, Object ctx) {
+        final CompletableFuture<Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo>>
+                ledgerInfosFuture = new CompletableFuture<>();
+        store.getManagedLedgerInfo(managedLedgerName, false, null,

Review Comment:
   @dlg99 This still confuse me. All data in `ManagedLedgerInfo` comes directly from `MLDataFormats.ManagedLedgerInfo.LedgerInfo`. I think it's better to just sync all the info to `ManagedLedgerInfo`.



-- 
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] tisonkun commented on pull request #15914: [Fix][Tiered Storage] Eagerly Delete Offloaded Segments On Topic Deletion

Posted by "tisonkun (via GitHub)" <gi...@apache.org>.
tisonkun commented on PR #15914:
URL: https://github.com/apache/pulsar/pull/15914#issuecomment-1447591061

   Remove the `release/*` tags since it changes the default behavior and should not go into early releases.


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