lhotari commented on code in PR #19394:
URL: https://github.com/apache/pulsar/pull/19394#discussion_r1094322490


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java:
##########
@@ -124,54 +124,72 @@ private synchronized long recoverBucketSnapshot() throws 
RuntimeException {
             }
         });
 
-        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+        Map<Range<Long>, ImmutableBucket> immutableBucketMap = 
immutableBuckets.asMapOfRanges();
+        if (immutableBucketMap.isEmpty()) {
             return 0;
         }
 
-        List<CompletableFuture<Void>> futures = new 
ArrayList<>(immutableBuckets.asMapOfRanges().size());
-        for (Map.Entry<Range<Long>, ImmutableBucket> entry 
:immutableBuckets.asMapOfRanges().entrySet()) {
+        Map<Range<Long>, CompletableFuture<List<DelayedIndex>>>
+                futures = new HashMap<>(immutableBucketMap.size());
+        for (Map.Entry<Range<Long>, ImmutableBucket> entry : 
immutableBucketMap.entrySet()) {
             Range<Long> key = entry.getKey();
             ImmutableBucket immutableBucket = entry.getValue();
-            CompletableFuture<Void> future =
-                    
immutableBucket.asyncRecoverBucketSnapshotEntry(this::getCutoffTime).thenAccept(indexList
 -> {
-                        if (CollectionUtils.isEmpty(indexList)) {
-                            // Delete bucket snapshot if indexList is empty
-                            toBeDeletedBucketMap.put(key, immutableBucket);
-                            return;
-                        }
-                        DelayedIndex lastDelayedIndex = 
indexList.get(indexList.size() - 1);
-                        synchronized (this.snapshotSegmentLastIndexTable) {
-                            
this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
-                                    lastDelayedIndex.getEntryId(), 
immutableBucket);
-                        }
-                        synchronized (this.sharedBucketPriorityQueue) {
-                            for (DelayedIndex index : indexList) {
-                                
this.sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
-                                        index.getEntryId());
-                            }
-                        }
-                    });
-            futures.add(future);
+            futures.put(key, 
immutableBucket.asyncRecoverBucketSnapshotEntry(this::getCutoffTime));
         }
 
         try {
-            FutureUtil.waitForAll(futures).whenComplete((__, ex) -> {
-                toBeDeletedBucketMap.forEach((k, immutableBucket) -> {
-                    immutableBuckets.asMapOfRanges().remove(k);
-                    immutableBucket.asyncDeleteBucketSnapshot();
-                });
-            }).get(AsyncOperationTimeoutSeconds, TimeUnit.SECONDS);
+            
FutureUtil.waitForAll(futures.values()).get(AsyncOperationTimeoutSeconds, 
TimeUnit.SECONDS);
         } catch (InterruptedException | ExecutionException | TimeoutException 
e) {
+            if (e instanceof InterruptedException) {
+                Thread.currentThread().interrupt();
+            }
             throw new RuntimeException(e);
         }
 
+        for (Map.Entry<Range<Long>, CompletableFuture<List<DelayedIndex>>> 
entry : futures.entrySet()) {
+            Range<Long> key = entry.getKey();
+            // the future will always be completed since it was waited for 
above
+            List<DelayedIndex> indexList = entry.getValue().getNow(null);
+            ImmutableBucket immutableBucket = immutableBucketMap.get(key);
+            if (CollectionUtils.isEmpty(indexList)) {
+                // Delete bucket snapshot if indexList is empty
+                toBeDeletedBucketMap.put(key, immutableBucket);
+            } else {
+                DelayedIndex lastDelayedIndex = indexList.get(indexList.size() 
- 1);
+                
this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                        lastDelayedIndex.getEntryId(), immutableBucket);
+                for (DelayedIndex index : indexList) {
+                    this.sharedBucketPriorityQueue.add(index.getTimestamp(), 
index.getLedgerId(),
+                            index.getEntryId());
+                }
+            }
+        }
+
+        List<CompletableFuture<Void>> deletionFutures = new ArrayList<>();
+        for (Map.Entry<Range<Long>, ImmutableBucket> mapEntry : 
toBeDeletedBucketMap.entrySet()) {
+            Range<Long> key = mapEntry.getKey();
+            ImmutableBucket immutableBucket = mapEntry.getValue();
+            immutableBucketMap.remove(key);
+            deletionFutures.add(immutableBucket.asyncDeleteBucketSnapshot());
+        }
+
+        // Wait for all deletion futures to complete before proceeding
+        try {
+            
FutureUtil.waitForAll(deletionFutures).get(AsyncOperationTimeoutSeconds, 
TimeUnit.SECONDS);
+        } catch (InterruptedException | ExecutionException | TimeoutException 
e) {
+            log.warn("asyncDeleteBucketSnapshot calls failed", e);
+            if (e instanceof InterruptedException) {
+                Thread.currentThread().interrupt();
+            }
+        }

Review Comment:
   @coderzc I removed the waiting. 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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to