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/10/14 01:38:12 UTC

[GitHub] [pulsar] coderzc opened a new pull request, #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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

   Master Issue: #16763 
   
   ### Motivation
   
   #16763 
   
   ### Modifications
   
   * Implement delayed message index bucket
   * Implement delayed message index bucket snapshot (create/load)
   * Reuse and refactor InMemoryDelayedDeliveryTracker
   * Reuse and refactor InMemoryDeliveryTrackerTest
   * Add test for BuketDelayedDeliveryTracker
   
   ### Documentation
   
   <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. -->
   
   - [ ] `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)
   
   Matching PR in forked repository
   PR in forked repository: https://github.com/coderzc/pulsar/pull/2
   


-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";

Review Comment:
   This seems to increase many objects, the `bucketKey` method will join the bucketKey.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {

Review Comment:
   see: https://github.com/apache/pulsar/pull/17611#discussion_r991790683



-- 
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] poorbarcode commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";

Review Comment:
   I notice that these two constants `DELIMITER` & `DELAYED_BUCKET_KEY_PREFIX` are only used in class `BucketState`. Why not declare them inside class `BucketState`?
   
   



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {

Review Comment:
   What does this parameter `isRecover` mean?  I think we should add documentation for it.
   
   



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Long> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(-1L));
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+                if (immutableBuckets.asMapOfRanges().size() > maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+            }
+        }
+
+        if (ledgerId < lastMutableBucketState.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucketState.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucketState.startLedgerId == -1L) {
+                lastMutableBucketState.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucketState.setEndLedgerId(ledgerId);
+        }
+
+        lastMutableBucketState.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {

Review Comment:
   We can simply replace these lines of code with the `nextDeliveryTime` method
   
   ```java
    boolean hasMessageAvailable = !getPriorityQueue().isEmpty() && getPriorityQueue().peekN1() <= cutoffTime;
   
   hasMessageAvailable = hasMessageAvailable
      || !sharedBucketPriorityQueue.isEmpty() && sharedBucketPriorityQueue.peekN1() <= cutoffTime;
   ```
   
   



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Long> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(-1L));

Review Comment:
   This `-1` doesn't really mean anything. I feel that a here is `CompletableFuture<Long>` misleading variable,  it should be declared `CompletableFuture<Void>`



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);

Review Comment:
   If we are sure to solve this problem in the future, should we add a `TODO` ?



-- 
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] poorbarcode commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();

Review Comment:
   I understand what you mean, `asyncCreateBucketSnapshot` is not a completely asynchronous method
   
   1. build bucket synchronously ( in synchronized block )
   2. persist asynchronously
   
   Should we change its name to `createBucketSnapshotAndAsyncPersistent`? Because two people have already misunderstood this method.



-- 
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] coderzc commented on pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load/recover) - part2

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

   Dependent on #17344 and #17164
   


-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(Bucket bucket, boolean isRebuild) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucket: {}", dispatcher.getName(), bucket);
+        }
+        if (bucket == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        final CompletableFuture<Long> createFuture = bucket.snapshotCreateFuture;
+        if (createFuture != null) {
+            // Wait bucket snapshot create finish
+            createFuture.get();
+        }
+
+        final String bucketKey = bucket.bucketKey();
+        final Long bucketId = getBucketIdByBucketKey(bucketKey);
+        Objects.requireNonNull(bucketId);
+
+        CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+        if (isRebuild) {
+            final long cutoffTime = getCutoffTime();
+            // Load Metadata of bucket snapshot
+            bucketSnapshotStorage.getBucketSnapshotMetadata(bucketId).thenAccept(snapshotMetadata -> {

Review Comment:
   Move to another PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] codecov-commenter commented on pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #17611:
URL: https://github.com/apache/pulsar/pull/17611#issuecomment-1278393736

   # [Codecov](https://codecov.io/gh/apache/pulsar/pull/17611?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#17611](https://codecov.io/gh/apache/pulsar/pull/17611?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d007cd8) into [master](https://codecov.io/gh/apache/pulsar/commit/6c65ca0d8a80bfaaa4d5869e0cea485f5c94369b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c65ca0) will **increase** coverage by `3.94%`.
   > The diff coverage is `0.62%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pulsar/pull/17611/graphs/tree.svg?width=650&height=150&src=pr&token=acYqCpsK9J&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pulsar/pull/17611?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #17611      +/-   ##
   ============================================
   + Coverage     34.91%   38.86%   +3.94%     
   - Complexity     5707     6251     +544     
   ============================================
     Files           607      609       +2     
     Lines         53396    53705     +309     
     Branches       5712     5752      +40     
   ============================================
   + Hits          18644    20873    +2229     
   + Misses        32119    30026    -2093     
   - Partials       2633     2806     +173     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `38.86% <0.62%> (+3.94%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pulsar/pull/17611?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...r/broker/delayed/BucketDelayedDeliveryTracker.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9kZWxheWVkL0J1Y2tldERlbGF5ZWREZWxpdmVyeVRyYWNrZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../org/apache/pulsar/broker/delayed/BucketState.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9kZWxheWVkL0J1Y2tldFN0YXRlLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...broker/delayed/InMemoryDelayedDeliveryTracker.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9kZWxheWVkL0luTWVtb3J5RGVsYXllZERlbGl2ZXJ5VHJhY2tlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [.../service/SystemTopicBasedTopicPoliciesService.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zZXJ2aWNlL1N5c3RlbVRvcGljQmFzZWRUb3BpY1BvbGljaWVzU2VydmljZS5qYXZh) | `51.26% <0.00%> (-0.33%)` | :arrow_down: |
   | [...sistent/PersistentDispatcherMultipleConsumers.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zZXJ2aWNlL3BlcnNpc3RlbnQvUGVyc2lzdGVudERpc3BhdGNoZXJNdWx0aXBsZUNvbnN1bWVycy5qYXZh) | `52.67% <0.00%> (+0.97%)` | :arrow_up: |
   | [...g/apache/pulsar/compaction/CompactedTopicImpl.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2NvbXBhY3Rpb24vQ29tcGFjdGVkVG9waWNJbXBsLmphdmE=) | `68.57% <0.00%> (+57.85%)` | :arrow_up: |
   | [...rg/apache/pulsar/broker/service/BrokerService.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zZXJ2aWNlL0Jyb2tlclNlcnZpY2UuamF2YQ==) | `49.28% <100.00%> (+1.28%)` | :arrow_up: |
   | [...a/org/apache/pulsar/io/file/FileListingThread.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWlvL2ZpbGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3B1bHNhci9pby9maWxlL0ZpbGVMaXN0aW5nVGhyZWFkLmphdmE=) | `80.24% <0.00%> (-2.47%)` | :arrow_down: |
   | [.../java/org/apache/pulsar/broker/web/WebService.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci93ZWIvV2ViU2VydmljZS5qYXZh) | `80.21% <0.00%> (-1.07%)` | :arrow_down: |
   | [...n/java/org/apache/pulsar/broker/PulsarService.java](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9QdWxzYXJTZXJ2aWNlLmphdmE=) | `54.20% <0.00%> (-0.12%)` | :arrow_down: |
   | ... and [92 more](https://codecov.io/gh/apache/pulsar/pull/17611/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);

Review Comment:
   See: https://github.com/apache/pulsar/pull/17611#discussion_r991948436



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] gaoran10 commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))

Review Comment:
   Can the future be an empty object? I'm not sure about the comment `Wait bucket snapshot create finish`.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";

Review Comment:
   Already move to `BucketState`.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));

Review Comment:
   We can record `bucketId` in the memory before operating the cursor property, when the broker restarted, the tracker rebuilds these indexes.



-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);

Review Comment:
   Ok, that make sense to me.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] gaoran10 commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))

Review Comment:
   Can the future be an empty object? I'm not sure about the command `Wait bucket snapshot create finish`



-- 
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] mattisonchao commented on pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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

   I will review it again tomorrow, thanks for your great work!


-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {
+                boolean exist = delayedIndexBitMap.containsKey(k);
+                byte[] bytes = v.toByteArray();
+                BitSet bitSet = BitSet.valueOf(bytes);
+                numberMessages.add(bitSet.cardinality());
+                if (!exist) {
+                    delayedIndexBitMap.put(k, bitSet);
+                } else {
+                    delayedIndexBitMap.get(k).or(bitSet);
+                }
+            });
+        }
+        return numberMessages.longValue();
+    }
+
+    boolean containsMessage(long ledgerId, int entryId) {
+        if (delayedIndexBitMap == null) {

Review Comment:
   I remove it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] codelipenghui merged pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;

Review Comment:
   It depends on your logic, exception/optioanl/-1 is ok for me.



-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {

Review Comment:
   Got it, thanks to you.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();

Review Comment:
   > this.numberDelayedMessages
   
   Ok.
   
   > If recoverBucketSnapshot throws an exception, the lastMutableBucket will be null.
   
   The `recoverBucketSnapshot` is in the constructor.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;

Review Comment:
   Return -1L?



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,573 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketId(BucketState bucketState) {
+        long bucketId = bucketState.getBucketId();
+        if (bucketId != -1L) {
+            return bucketId;
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return -1L;
+        }
+
+        bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createImmutableBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenApply(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    });
+                    return newBucketId;
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Long> snapshotCreateFuture = bucketState.snapshotCreateFuture;
+        if (snapshotCreateFuture == null) {
+            snapshotCreateFuture = CompletableFuture.completedFuture(-1L);
+        }
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final Long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();

Review Comment:
   All public methods of the tracker already add `synchronized` to ensure synchronization and without race conditions in the callback logic.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));

Review Comment:
   Ok, I improve it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {

Review Comment:
   The `isRecover` is `true` means is the method used to recover bucket snapshot when the broker restarts, it will load bucket metadata to memory and some other special handling.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();

Review Comment:
   The `addMessage` will be synchronous by `synchronized`, and all possible conflict operations in `asyncCreateBucketSnapshot` also are synchronous.



-- 
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] coderzc commented on pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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

   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));

Review Comment:
   The `immutableBuckets.span()` can reduce some unnecessary search operations.



-- 
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] coderzc commented on pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load/recover) - part2

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

   /pulsarbot rerun-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();

Review Comment:
   the reason is the same as below.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+                if (immutableBuckets.asMapOfRanges().size() > maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+            }
+        }
+
+        if (ledgerId < lastMutableBucketState.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucketState.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucketState.startLedgerId == -1L) {
+                lastMutableBucketState.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucketState.setEndLedgerId(ledgerId);
+        }
+
+        lastMutableBucketState.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {
+        long cutoffTime = getCutoffTime();
+
+        boolean hasMessageAvailable = getNumberOfDelayedMessages() > 0 && nextDeliveryTime() <= cutoffTime;
+        if (!hasMessageAvailable) {
+            updateTimer();
+        }
+        return hasMessageAvailable;
+    }
+
+    @Override
+    protected long nextDeliveryTime() {
+        if (getPriorityQueue().isEmpty() && !sharedBucketPriorityQueue.isEmpty()) {
+            return sharedBucketPriorityQueue.peekN1();
+        } else if (sharedBucketPriorityQueue.isEmpty() && !getPriorityQueue().isEmpty()) {
+            return getPriorityQueue().peekN1();
+        }
+        long timestamp = getPriorityQueue().peekN1();
+        long bucketTimestamp = sharedBucketPriorityQueue.peekN1();
+        return Math.min(timestamp, bucketTimestamp);
+    }
+
+    @Override
+    public synchronized long getNumberOfDelayedMessages() {
+        return numberDelayedMessages;
+    }
+
+    @Override
+    public synchronized long getBufferMemoryUsage() {
+        return getPriorityQueue().bytesCapacity() + sharedBucketPriorityQueue.bytesCapacity();
+    }
+
+    @Override
+    public synchronized Set<PositionImpl> getScheduledMessages(int maxMessages) {
+        long cutoffTime = getCutoffTime();
+
+        moveScheduledMessageToSharedQueue(cutoffTime);
+
+        Set<PositionImpl> positions = new TreeSet<>();
+        int n = maxMessages;
+
+        while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) {
+            long timestamp = sharedBucketPriorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = sharedBucketPriorityQueue.peekN2();
+            long entryId = sharedBucketPriorityQueue.peekN3();
+            positions.add(new PositionImpl(ledgerId, entryId));
+
+            sharedBucketPriorityQueue.pop();
+            removeIndexBit(ledgerId, entryId);
+
+            BucketState bucketState = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
+            if (bucketState != null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Load next snapshot segment, bucketState: {}", dispatcher.getName(), bucketState);
+                }
+                // All message of current snapshot segment are scheduled, load next snapshot segment
+                // TODO make it asynchronous and not blocking this process
+                try {
+                    asyncLoadNextBucketSnapshotEntry(bucketState, false).get(AsyncOperationTimeoutSeconds,
+                            TimeUnit.SECONDS);
+                } catch (InterruptedException | ExecutionException | TimeoutException e) {
+                    throw new RuntimeException(e);

Review Comment:
   Ok.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load/recover) - part2

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

   @codelipenghui @gaoran10 @Technoboy- 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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,572 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketId(BucketState bucketState) {
+        long bucketId = bucketState.getBucketId();
+        if (bucketId != -1L) {

Review Comment:
   OK



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();

Review Comment:
   I'm not sure If we exist some race condition cause poped entry is not the peeked entry. 
   e.g. After the peek, another thread popped up with this entry.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;

Review Comment:
   Maybe we can easily return `newBucketId` here.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();

Review Comment:
   I'm not sure If we exist some race condition cause poped entry is not the peeked entry.
   e.g. After the peek, another thread popped up with this entry.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);

Review Comment:
   Should we give it an optional Long? Because the id of the bucket is empty.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;

Review Comment:
   In the `exceptionally`? this only is an exceptional case.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);

Review Comment:
   I will ensure they will be handled.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,573 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketId(BucketState bucketState) {
+        long bucketId = bucketState.getBucketId();
+        if (bucketId != -1L) {
+            return bucketId;
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return -1L;
+        }
+
+        bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createImmutableBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenApply(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {

Review Comment:
   I won't make `putBucketKeyId` method asynchronous execution, when it failed the tracker can still continue to run.



-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketState.java:
##########
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+@Data
+@AllArgsConstructor
+public class BucketState {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    long bucketId;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;

Review Comment:
   Adding the `getter` method return `Optional` will reduce the NPE risk.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,573 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketId(BucketState bucketState) {
+        long bucketId = bucketState.getBucketId();
+        if (bucketId != -1L) {
+            return bucketId;
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return -1L;
+        }
+
+        bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createImmutableBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenApply(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {

Review Comment:
   You have to return 'CompletionStage' for asynchronous operations to make it run sequentially.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,573 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketId(BucketState bucketState) {
+        long bucketId = bucketState.getBucketId();
+        if (bucketId != -1L) {
+            return bucketId;
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return -1L;
+        }
+
+        bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createImmutableBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenApply(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    });
+                    return newBucketId;
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createImmutableBucket(startLedgerId, endLedgerId);

Review Comment:
   I think we have to change this method name because it looks mutable.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,572 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketId(BucketState bucketState) {
+        long bucketId = bucketState.getBucketId();
+        if (bucketId != -1L) {

Review Comment:
   If you want `-1` as a particular value, maybe we must make it constant like DELAYED_BUCKET_KEY_PREFIX



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,573 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketId(BucketState bucketState) {
+        long bucketId = bucketState.getBucketId();
+        if (bucketId != -1L) {
+            return bucketId;
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return -1L;
+        }
+
+        bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createImmutableBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenApply(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    });
+                    return newBucketId;
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Long> snapshotCreateFuture = bucketState.snapshotCreateFuture;
+        if (snapshotCreateFuture == null) {
+            snapshotCreateFuture = CompletableFuture.completedFuture(-1L);
+        }
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final Long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();

Review Comment:
   This method may invoke concurrently, do we have any risk for race conditions here?



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {

Review Comment:
   I will remove it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(Bucket bucket, boolean isRebuild) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucket: {}", dispatcher.getName(), bucket);
+        }
+        if (bucket == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        final CompletableFuture<Long> createFuture = bucket.snapshotCreateFuture;
+        if (createFuture != null) {
+            // Wait bucket snapshot create finish
+            createFuture.get();

Review Comment:
   Ok.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)

Review Comment:
   It has at least one element.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java:
##########
@@ -274,22 +279,29 @@ public void run(Timeout timeout) throws Exception {
 
     @Override
     public void close() {
-        priorityQueue.close();
         if (timeout != null) {
             timeout.cancel();
         }
+        priorityQueue.close();

Review Comment:
   I push a new PR to fix it. #18000



-- 
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] coderzc closed pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load/recover) - part2

Posted by GitBox <gi...@apache.org>.
coderzc closed pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load/recover) - part2
URL: https://github.com/apache/pulsar/pull/17611


-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();

Review Comment:
   The `addMessage` will be synchronous by synchronized, and all possible conflict operations in `asyncCreateBucketSnapshot` also are synchronous.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),

Review Comment:
   I think can add parameter `maxRetryTimes`  in `executeWithRetry`.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+                if (immutableBuckets.asMapOfRanges().size() > maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+            }
+        }
+
+        if (ledgerId < lastMutableBucketState.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucketState.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucketState.startLedgerId == -1L) {
+                lastMutableBucketState.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucketState.setEndLedgerId(ledgerId);
+        }
+
+        lastMutableBucketState.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {
+        long cutoffTime = getCutoffTime();
+
+        boolean hasMessageAvailable = getNumberOfDelayedMessages() > 0 && nextDeliveryTime() <= cutoffTime;
+        if (!hasMessageAvailable) {
+            updateTimer();
+        }
+        return hasMessageAvailable;
+    }
+
+    @Override
+    protected long nextDeliveryTime() {
+        if (getPriorityQueue().isEmpty() && !sharedBucketPriorityQueue.isEmpty()) {
+            return sharedBucketPriorityQueue.peekN1();
+        } else if (sharedBucketPriorityQueue.isEmpty() && !getPriorityQueue().isEmpty()) {
+            return getPriorityQueue().peekN1();
+        }
+        long timestamp = getPriorityQueue().peekN1();
+        long bucketTimestamp = sharedBucketPriorityQueue.peekN1();
+        return Math.min(timestamp, bucketTimestamp);
+    }
+
+    @Override
+    public synchronized long getNumberOfDelayedMessages() {
+        return numberDelayedMessages;
+    }
+
+    @Override
+    public synchronized long getBufferMemoryUsage() {
+        return getPriorityQueue().bytesCapacity() + sharedBucketPriorityQueue.bytesCapacity();
+    }
+
+    @Override
+    public synchronized Set<PositionImpl> getScheduledMessages(int maxMessages) {
+        long cutoffTime = getCutoffTime();
+
+        moveScheduledMessageToSharedQueue(cutoffTime);
+
+        Set<PositionImpl> positions = new TreeSet<>();
+        int n = maxMessages;
+
+        while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) {
+            long timestamp = sharedBucketPriorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = sharedBucketPriorityQueue.peekN2();
+            long entryId = sharedBucketPriorityQueue.peekN3();
+            positions.add(new PositionImpl(ledgerId, entryId));
+
+            sharedBucketPriorityQueue.pop();
+            removeIndexBit(ledgerId, entryId);
+
+            BucketState bucketState = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
+            if (bucketState != null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Load next snapshot segment, bucketState: {}", dispatcher.getName(), bucketState);
+                }
+                // All message of current snapshot segment are scheduled, load next snapshot segment
+                // TODO make it asynchronous and not blocking this process
+                try {
+                    asyncLoadNextBucketSnapshotEntry(bucketState, false).get(AsyncOperationTimeoutSeconds,
+                            TimeUnit.SECONDS);
+                } catch (InterruptedException | ExecutionException | TimeoutException e) {
+                    throw new RuntimeException(e);

Review Comment:
   I will improve it by making it asynchronous in the future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,567 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {

Review Comment:
   I‘m not sure if this will break some previous behaviors, if 'messagesHaveFixedDelay' only used
   to limit the overinflated memory footprint,we can don't need to care about it. /cc @codelipenghui 



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();

Review Comment:
   > If recoverBucketSnapshot throws an exception, the lastMutableBucket will be null.
   
   The `recoverBucketSnapshot` is in the constructor.



-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));

Review Comment:
   Make sense to me.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());

Review Comment:
   Ok, I agree with it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);

Review Comment:
   Ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();

Review Comment:
   Same for https://github.com/apache/pulsar/pull/17611#discussion_r1003065822



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);

Review Comment:
   In fact, the return value has not been used.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows

Review Comment:
   I remove it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+                if (immutableBuckets.asMapOfRanges().size() > maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+            }
+        }
+
+        if (ledgerId < lastMutableBucketState.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucketState.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucketState.startLedgerId == -1L) {
+                lastMutableBucketState.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucketState.setEndLedgerId(ledgerId);
+        }
+
+        lastMutableBucketState.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {
+        long cutoffTime = getCutoffTime();
+
+        boolean hasMessageAvailable = getNumberOfDelayedMessages() > 0 && nextDeliveryTime() <= cutoffTime;
+        if (!hasMessageAvailable) {
+            updateTimer();
+        }
+        return hasMessageAvailable;
+    }
+
+    @Override
+    protected long nextDeliveryTime() {
+        if (getPriorityQueue().isEmpty() && !sharedBucketPriorityQueue.isEmpty()) {
+            return sharedBucketPriorityQueue.peekN1();
+        } else if (sharedBucketPriorityQueue.isEmpty() && !getPriorityQueue().isEmpty()) {
+            return getPriorityQueue().peekN1();
+        }
+        long timestamp = getPriorityQueue().peekN1();
+        long bucketTimestamp = sharedBucketPriorityQueue.peekN1();
+        return Math.min(timestamp, bucketTimestamp);
+    }
+
+    @Override
+    public synchronized long getNumberOfDelayedMessages() {
+        return numberDelayedMessages;
+    }
+
+    @Override
+    public synchronized long getBufferMemoryUsage() {
+        return getPriorityQueue().bytesCapacity() + sharedBucketPriorityQueue.bytesCapacity();
+    }
+
+    @Override
+    public synchronized Set<PositionImpl> getScheduledMessages(int maxMessages) {
+        long cutoffTime = getCutoffTime();
+
+        moveScheduledMessageToSharedQueue(cutoffTime);
+
+        Set<PositionImpl> positions = new TreeSet<>();
+        int n = maxMessages;
+
+        while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) {
+            long timestamp = sharedBucketPriorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = sharedBucketPriorityQueue.peekN2();
+            long entryId = sharedBucketPriorityQueue.peekN3();
+            positions.add(new PositionImpl(ledgerId, entryId));
+
+            sharedBucketPriorityQueue.pop();
+            removeIndexBit(ledgerId, entryId);
+
+            BucketState bucketState = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
+            if (bucketState != null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Load next snapshot segment, bucketState: {}", dispatcher.getName(), bucketState);
+                }
+                // All message of current snapshot segment are scheduled, load next snapshot segment
+                // TODO make it asynchronous and not blocking this process
+                try {
+                    asyncLoadNextBucketSnapshotEntry(bucketState, false).get(AsyncOperationTimeoutSeconds,
+                            TimeUnit.SECONDS);
+                } catch (InterruptedException | ExecutionException | TimeoutException e) {
+                    throw new RuntimeException(e);

Review Comment:
   I will improve it in the future by make it asynchronous.



-- 
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] poorbarcode commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+                if (immutableBuckets.asMapOfRanges().size() > maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+            }
+        }
+
+        if (ledgerId < lastMutableBucketState.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucketState.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucketState.startLedgerId == -1L) {
+                lastMutableBucketState.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucketState.setEndLedgerId(ledgerId);
+        }
+
+        lastMutableBucketState.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {
+        long cutoffTime = getCutoffTime();
+
+        boolean hasMessageAvailable = getNumberOfDelayedMessages() > 0 && nextDeliveryTime() <= cutoffTime;
+        if (!hasMessageAvailable) {
+            updateTimer();
+        }
+        return hasMessageAvailable;
+    }
+
+    @Override
+    protected long nextDeliveryTime() {
+        if (getPriorityQueue().isEmpty() && !sharedBucketPriorityQueue.isEmpty()) {
+            return sharedBucketPriorityQueue.peekN1();
+        } else if (sharedBucketPriorityQueue.isEmpty() && !getPriorityQueue().isEmpty()) {
+            return getPriorityQueue().peekN1();
+        }
+        long timestamp = getPriorityQueue().peekN1();
+        long bucketTimestamp = sharedBucketPriorityQueue.peekN1();
+        return Math.min(timestamp, bucketTimestamp);
+    }
+
+    @Override
+    public synchronized long getNumberOfDelayedMessages() {
+        return numberDelayedMessages;
+    }
+
+    @Override
+    public synchronized long getBufferMemoryUsage() {
+        return getPriorityQueue().bytesCapacity() + sharedBucketPriorityQueue.bytesCapacity();
+    }
+
+    @Override
+    public synchronized Set<PositionImpl> getScheduledMessages(int maxMessages) {
+        long cutoffTime = getCutoffTime();
+
+        moveScheduledMessageToSharedQueue(cutoffTime);
+
+        Set<PositionImpl> positions = new TreeSet<>();
+        int n = maxMessages;
+
+        while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) {
+            long timestamp = sharedBucketPriorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = sharedBucketPriorityQueue.peekN2();
+            long entryId = sharedBucketPriorityQueue.peekN3();
+            positions.add(new PositionImpl(ledgerId, entryId));
+
+            sharedBucketPriorityQueue.pop();
+            removeIndexBit(ledgerId, entryId);
+
+            BucketState bucketState = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
+            if (bucketState != null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Load next snapshot segment, bucketState: {}", dispatcher.getName(), bucketState);
+                }
+                // All message of current snapshot segment are scheduled, load next snapshot segment
+                // TODO make it asynchronous and not blocking this process
+                try {
+                    asyncLoadNextBucketSnapshotEntry(bucketState, false).get(AsyncOperationTimeoutSeconds,
+                            TimeUnit.SECONDS);
+                } catch (InterruptedException | ExecutionException | TimeoutException e) {
+                    throw new RuntimeException(e);

Review Comment:
   Should we add a `TODO` here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] gaoran10 commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);

Review Comment:
   This is a warning log, if failed to record the bucket id in the cursor's property, what's the problem it will cause? Do we need to clean up the snapshot?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),

Review Comment:
   I'm not sure we can always try to modify the property when encountering `ManagedLedgerException.BadVersionException`, in common, it's ok, I'm afraid it's become a dead cycle, maybe when the data version of the cursor lower than the metadata store, this will happen, maybe this is another problem.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))

Review Comment:
   Can the future be an empty object?



-- 
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] poorbarcode commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {

Review Comment:
   OK



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] gaoran10 commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),

Review Comment:
   I'm not sure we can always try to modify the property when encountering `ManagedLedgerException.BadVersionException`, in common, it's ok, I'm afraid it's become a dead cycle when the data version of the cursor lower than the metadata store, this will happen, maybe this is another problem.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));

Review Comment:
   We can record `bucketId` in the memory before operating the cursor property, and when the broker restarted, the tracker rebuilds these indexes.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {
+                boolean exist = delayedIndexBitMap.containsKey(k);
+                byte[] bytes = v.toByteArray();
+                BitSet bitSet = BitSet.valueOf(bytes);
+                numberMessages.add(bitSet.cardinality());
+                if (!exist) {
+                    delayedIndexBitMap.put(k, bitSet);
+                } else {
+                    delayedIndexBitMap.get(k).or(bitSet);
+                }
+            });
+        }
+        return numberMessages.longValue();
+    }
+
+    boolean containsMessage(long ledgerId, int entryId) {
+        if (delayedIndexBitMap == null) {
+            return false;
+        }
+
+        BitSet bitSet = delayedIndexBitMap.get(ledgerId);
+        if (bitSet == null) {
+            return false;
+        }
+        return bitSet.get(entryId);
+    }
+
+    void putIndexBit(long ledgerId, long entryId) {
+        if (entryId < Integer.MAX_VALUE) {
+            delayedIndexBitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);

Review Comment:
   Yes, you are right.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,573 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketId(BucketState bucketState) {
+        long bucketId = bucketState.getBucketId();
+        if (bucketId != -1L) {
+            return bucketId;
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return -1L;
+        }
+
+        bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createImmutableBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenApply(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {

Review Comment:
   I won't make `putBucketKeyId` method asynchronous execution, when it failed the tracker can still continue to run.



-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Long> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(-1L));
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+                if (immutableBuckets.asMapOfRanges().size() > maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+            }
+        }
+
+        if (ledgerId < lastMutableBucketState.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucketState.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucketState.startLedgerId == -1L) {
+                lastMutableBucketState.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucketState.setEndLedgerId(ledgerId);
+        }
+
+        lastMutableBucketState.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {
+        long cutoffTime = getCutoffTime();
+
+        boolean hasMessageAvailable = !getPriorityQueue().isEmpty() && getPriorityQueue().peekN1() <= cutoffTime;
+
+        hasMessageAvailable = hasMessageAvailable
+                || !sharedBucketPriorityQueue.isEmpty() && sharedBucketPriorityQueue.peekN1() <= cutoffTime;
+        if (!hasMessageAvailable) {
+            updateTimer();
+        }
+        return hasMessageAvailable;
+    }
+
+    @Override
+    protected long nextDeliveryTime() {
+        if (getPriorityQueue().isEmpty() && !sharedBucketPriorityQueue.isEmpty()) {
+            return sharedBucketPriorityQueue.peekN1();
+        } else if (sharedBucketPriorityQueue.isEmpty() && !getPriorityQueue().isEmpty()) {
+            return getPriorityQueue().peekN1();
+        }
+        long timestamp = getPriorityQueue().peekN1();
+        long bucketTimestamp = sharedBucketPriorityQueue.peekN1();
+        return Math.min(timestamp, bucketTimestamp);
+    }
+
+    @Override
+    public synchronized long getNumberOfDelayedMessages() {
+        return numberDelayedMessages;
+    }
+
+    @Override
+    public synchronized long getBufferMemoryUsage() {
+        return getPriorityQueue().bytesCapacity() + sharedBucketPriorityQueue.bytesCapacity();
+    }
+
+    @Override
+    public synchronized Set<PositionImpl> getScheduledMessages(int maxMessages) {
+        long cutoffTime = getCutoffTime();
+
+        moveScheduledMessageToSharedQueue(cutoffTime);
+
+        Set<PositionImpl> positions = new TreeSet<>();
+        int n = maxMessages;
+
+        while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) {
+            long timestamp = sharedBucketPriorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = sharedBucketPriorityQueue.peekN2();
+            long entryId = sharedBucketPriorityQueue.peekN3();
+            positions.add(new PositionImpl(ledgerId, entryId));
+
+            sharedBucketPriorityQueue.pop();
+            removeIndexBit(ledgerId, entryId);
+
+            BucketState bucketState = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
+            if (bucketState != null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Load next snapshot segment, bucketState: {}", dispatcher.getName(), bucketState);
+                }
+                // All message of current snapshot segment are scheduled, load next snapshot segment
+                // TODO make it asynchronous
+                try {
+                    asyncLoadNextBucketSnapshotEntry(bucketState, false).get(AsyncOperationTimeoutSeconds,

Review Comment:
   We have to mention that we may make it not blocking in the next part. Right?



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Long> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(-1L));
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+                if (immutableBuckets.asMapOfRanges().size() > maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+            }
+        }
+
+        if (ledgerId < lastMutableBucketState.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucketState.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucketState.startLedgerId == -1L) {
+                lastMutableBucketState.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucketState.setEndLedgerId(ledgerId);
+        }
+
+        lastMutableBucketState.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {
+        long cutoffTime = getCutoffTime();
+
+        boolean hasMessageAvailable = !getPriorityQueue().isEmpty() && getPriorityQueue().peekN1() <= cutoffTime;
+
+        hasMessageAvailable = hasMessageAvailable
+                || !sharedBucketPriorityQueue.isEmpty() && sharedBucketPriorityQueue.peekN1() <= cutoffTime;
+        if (!hasMessageAvailable) {
+            updateTimer();
+        }
+        return hasMessageAvailable;
+    }
+
+    @Override
+    protected long nextDeliveryTime() {
+        if (getPriorityQueue().isEmpty() && !sharedBucketPriorityQueue.isEmpty()) {
+            return sharedBucketPriorityQueue.peekN1();
+        } else if (sharedBucketPriorityQueue.isEmpty() && !getPriorityQueue().isEmpty()) {
+            return getPriorityQueue().peekN1();
+        }
+        long timestamp = getPriorityQueue().peekN1();
+        long bucketTimestamp = sharedBucketPriorityQueue.peekN1();
+        return Math.min(timestamp, bucketTimestamp);
+    }
+
+    @Override
+    public synchronized long getNumberOfDelayedMessages() {
+        return numberDelayedMessages;
+    }
+
+    @Override
+    public synchronized long getBufferMemoryUsage() {
+        return getPriorityQueue().bytesCapacity() + sharedBucketPriorityQueue.bytesCapacity();
+    }
+
+    @Override
+    public synchronized Set<PositionImpl> getScheduledMessages(int maxMessages) {
+        long cutoffTime = getCutoffTime();
+
+        moveScheduledMessageToSharedQueue(cutoffTime);
+
+        Set<PositionImpl> positions = new TreeSet<>();
+        int n = maxMessages;
+
+        while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) {
+            long timestamp = sharedBucketPriorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = sharedBucketPriorityQueue.peekN2();
+            long entryId = sharedBucketPriorityQueue.peekN3();
+            positions.add(new PositionImpl(ledgerId, entryId));
+
+            sharedBucketPriorityQueue.pop();
+            removeIndexBit(ledgerId, entryId);
+
+            BucketState bucketState = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
+            if (bucketState != null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Load next snapshot segment, bucketState: {}", dispatcher.getName(), bucketState);
+                }
+                // All message of current snapshot segment are scheduled, load next snapshot segment
+                // TODO make it asynchronous
+                try {
+                    asyncLoadNextBucketSnapshotEntry(bucketState, false).get(AsyncOperationTimeoutSeconds,

Review Comment:
   Yes.



-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;

Review Comment:
   Oh, I got it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);

Review Comment:
   The `Bucket` object only records some states, maybe it should be called `BucketState`.



-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java:
##########
@@ -274,22 +279,29 @@ public void run(Timeout timeout) throws Exception {
 
     @Override
     public void close() {
-        priorityQueue.close();
         if (timeout != null) {
             timeout.cancel();
         }
+        priorityQueue.close();

Review Comment:
   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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);

Review Comment:
   The `Bucket` only record some state, maybe it should be called `BucketState`.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);

Review Comment:
   The `Bucket` only records some state, maybe it should be called `BucketState`.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,572 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketId(BucketState bucketState) {
+        long bucketId = bucketState.getBucketId();
+        if (bucketId != -1L) {

Review Comment:
   In fact `-1` is a zero value, maybe it should be the same as `snapshotCreateFuture` to return `Optional`.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketState.java:
##########
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+@Data
+@AllArgsConstructor
+public class BucketState {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    long bucketId;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;

Review Comment:
   Ok.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] Technoboy- commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on code in PR #17611:
URL: https://github.com/apache/pulsar/pull/17611#discussion_r997672583


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));

Review Comment:
   Return this directly ?



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));

Review Comment:
   The `immutableBuckets.span()` can reduce some unnecessary search operations.



-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java:
##########
@@ -274,22 +279,29 @@ public void run(Timeout timeout) throws Exception {
 
     @Override
     public void close() {
-        priorityQueue.close();
         if (timeout != null) {
             timeout.cancel();
         }
+        priorityQueue.close();

Review Comment:
   If it's a problem, we can push a new fix. Because it can help cherry-pick another branch.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java:
##########
@@ -274,22 +279,29 @@ public void run(Timeout timeout) throws Exception {
 
     @Override
     public void close() {
-        priorityQueue.close();
         if (timeout != null) {
             timeout.cancel();
         }
+        priorityQueue.close();

Review Comment:
   If it's a problem, we can push a new fix. Because it can help cherry-pick to another branch.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();

Review Comment:
   Move to another PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] mattisonchao closed pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

Posted by GitBox <gi...@apache.org>.
mattisonchao closed pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2
URL: https://github.com/apache/pulsar/pull/17611


-- 
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] poorbarcode commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,567 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {

Review Comment:
   The variables `fixedDelayDetectionLookahead`, `messagesHaveFixedDelay`, and `highestDeliveryTimeTracked` are mainly used to limit the overinflated memory footprint, which `PIP-195` has solved. We can rewrite method `shouldPauseAllDeliveries` in `BucketDelayedDeliveryTracker ` like this, right?
   
   ```java
   public boolean shouldPauseAllDeliveries() {
       return false;
   }
   ```
   
   Or rewrite like this:
   ```java
   public boolean shouldPauseAllDeliveries() {
       return sharedBucketPriorityQueue.size() > fixedDelayDetectionLookahead;
   }
   ```
   
   And then in class `BucketDelayedDeliveryTracker` we don't need to care about variables `messagesHaveFixedDelay`



-- 
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] poorbarcode commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();

Review Comment:
   How can I prevent concurrent calls to `asyncCreateBucketSnapshot`? 
   
   E.g:
   The config is:
   ```
   minIndexCountPerBucket = 5000
   ```
   At the current time, `priorityQueue.size() = 4999`, and the max ledger id is `4999` in the queue.
   - add one message(5000:1)
   - trigger `asyncCreateBucketSnapshot`
   - add the second message(5001:1)
   - trigger `asyncCreateBucketSnapshot` twice.
   



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Long> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(-1L));
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {

Review Comment:
   Why is this condition `|| existBucket`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();

Review Comment:
   Why mark this comment as resolved?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();

Review Comment:
   Should you wait for the asynchronous task `asyncCreateBucketSnapshot ` to complete before executing `resetLastMutableBucketRange`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+                if (immutableBuckets.asMapOfRanges().size() > maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+            }
+        }
+
+        if (ledgerId < lastMutableBucketState.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucketState.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucketState.startLedgerId == -1L) {
+                lastMutableBucketState.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucketState.setEndLedgerId(ledgerId);
+        }
+
+        lastMutableBucketState.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {
+        long cutoffTime = getCutoffTime();
+
+        boolean hasMessageAvailable = getNumberOfDelayedMessages() > 0 && nextDeliveryTime() <= cutoffTime;
+        if (!hasMessageAvailable) {
+            updateTimer();
+        }
+        return hasMessageAvailable;
+    }
+
+    @Override
+    protected long nextDeliveryTime() {
+        if (getPriorityQueue().isEmpty() && !sharedBucketPriorityQueue.isEmpty()) {
+            return sharedBucketPriorityQueue.peekN1();
+        } else if (sharedBucketPriorityQueue.isEmpty() && !getPriorityQueue().isEmpty()) {
+            return getPriorityQueue().peekN1();
+        }
+        long timestamp = getPriorityQueue().peekN1();
+        long bucketTimestamp = sharedBucketPriorityQueue.peekN1();
+        return Math.min(timestamp, bucketTimestamp);
+    }
+
+    @Override
+    public synchronized long getNumberOfDelayedMessages() {
+        return numberDelayedMessages;
+    }
+
+    @Override
+    public synchronized long getBufferMemoryUsage() {
+        return getPriorityQueue().bytesCapacity() + sharedBucketPriorityQueue.bytesCapacity();
+    }
+
+    @Override
+    public synchronized Set<PositionImpl> getScheduledMessages(int maxMessages) {
+        long cutoffTime = getCutoffTime();
+
+        moveScheduledMessageToSharedQueue(cutoffTime);
+
+        Set<PositionImpl> positions = new TreeSet<>();
+        int n = maxMessages;
+
+        while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) {
+            long timestamp = sharedBucketPriorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = sharedBucketPriorityQueue.peekN2();
+            long entryId = sharedBucketPriorityQueue.peekN3();
+            positions.add(new PositionImpl(ledgerId, entryId));
+
+            sharedBucketPriorityQueue.pop();
+            removeIndexBit(ledgerId, entryId);
+
+            BucketState bucketState = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
+            if (bucketState != null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Load next snapshot segment, bucketState: {}", dispatcher.getName(), bucketState);
+                }
+                // All message of current snapshot segment are scheduled, load next snapshot segment
+                // TODO make it asynchronous and not blocking this process
+                try {
+                    asyncLoadNextBucketSnapshotEntry(bucketState, false).get(AsyncOperationTimeoutSeconds,
+                            TimeUnit.SECONDS);
+                } catch (InterruptedException | ExecutionException | TimeoutException e) {
+                    throw new RuntimeException(e);

Review Comment:
   If an exception occurs here, the next segment will never load 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.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();

Review Comment:
   Same for https://github.com/apache/pulsar/pull/17611#discussion_r995249586



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,567 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {

Review Comment:
   I‘m not sure if this will break some previous behaviors, if 'messagesHaveFixedDelay' only used to limit the overinflated memory footprint,we can don't need to care about it. /cc @codelipenghui 



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,567 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {

Review Comment:
   I‘m not sure if this will break some previous behaviors, if 'messagesHaveFixedDelay' only used to limit the overinflated memory footprint, we can don't need to care about it. /cc @codelipenghui 



-- 
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] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {

Review Comment:
   Maybe we can create two classes, one is `ImmutableBucket` and another is `MutableBucket`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";

Review Comment:
   What do you think to make the `bucketKey` to be a class?
   ```java
   class BucketKey {
      private static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
      private static final String DELIMITER = "_";
      private final long ledgerId;
   
      // provide some method to get the ledger id
   }
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();

Review Comment:
   Can it be given an initial value to avoid expansion?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;

Review Comment:
   I'm not sure if it is good to return null. because it has a risk to involve NPE.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows

Review Comment:
   Why `@SneakyThrows`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(Bucket bucket, boolean isRebuild) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucket: {}", dispatcher.getName(), bucket);
+        }
+        if (bucket == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        final CompletableFuture<Long> createFuture = bucket.snapshotCreateFuture;
+        if (createFuture != null) {
+            // Wait bucket snapshot create finish
+            createFuture.get();
+        }
+
+        final String bucketKey = bucket.bucketKey();
+        final Long bucketId = getBucketIdByBucketKey(bucketKey);
+        Objects.requireNonNull(bucketId);
+
+        CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+        if (isRebuild) {
+            final long cutoffTime = getCutoffTime();
+            // Load Metadata of bucket snapshot
+            bucketSnapshotStorage.getBucketSnapshotMetadata(bucketId).thenAccept(snapshotMetadata -> {

Review Comment:
   If this method got any exception, `loadMetaDataFuture` will not be complete.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(Bucket bucket, boolean isRebuild) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucket: {}", dispatcher.getName(), bucket);
+        }
+        if (bucket == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        final CompletableFuture<Long> createFuture = bucket.snapshotCreateFuture;
+        if (createFuture != null) {
+            // Wait bucket snapshot create finish
+            createFuture.get();
+        }
+
+        final String bucketKey = bucket.bucketKey();
+        final Long bucketId = getBucketIdByBucketKey(bucketKey);
+        Objects.requireNonNull(bucketId);
+
+        CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+        if (isRebuild) {
+            final long cutoffTime = getCutoffTime();
+            // Load Metadata of bucket snapshot
+            bucketSnapshotStorage.getBucketSnapshotMetadata(bucketId).thenAccept(snapshotMetadata -> {
+                List<SnapshotSegmentMetadata> metadataList = snapshotMetadata.getMetadataListList();
+
+                // Skip all already reach schedule time snapshot segments
+                int nextSnapshotEntryIndex = 0;
+                while (nextSnapshotEntryIndex < metadataList.size()
+                        && metadataList.get(nextSnapshotEntryIndex).getMaxScheduleTimestamp() <= cutoffTime) {
+                    nextSnapshotEntryIndex++;
+                }
+
+                final int lastSegmentEntryId = metadataList.size();
+
+                long numberMessages = bucket.covertDelayIndexMapAndCount(nextSnapshotEntryIndex, metadataList);

Review Comment:
   It's an immutable bucket. Why can we modify it?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));

Review Comment:
   If the retry operation failed, we may lose a bucket.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {
+                boolean exist = delayedIndexBitMap.containsKey(k);
+                byte[] bytes = v.toByteArray();
+                BitSet bitSet = BitSet.valueOf(bytes);
+                numberMessages.add(bitSet.cardinality());
+                if (!exist) {
+                    delayedIndexBitMap.put(k, bitSet);
+                } else {
+                    delayedIndexBitMap.get(k).or(bitSet);
+                }
+            });
+        }
+        return numberMessages.longValue();
+    }
+
+    boolean containsMessage(long ledgerId, int entryId) {
+        if (delayedIndexBitMap == null) {

Review Comment:
   Why we can allow users to put the map to null?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());

Review Comment:
   Do you consider moving` new HashMap<>()` to the construction method? because I don't see any other use cases.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {

Review Comment:
   Would you like the `BucketKey` class?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(Bucket bucket, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucket: {}", dispatcher.getName(), bucket);
+        }
+        if (bucket == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        final CompletableFuture<Long> createFuture = bucket.snapshotCreateFuture;
+        if (createFuture != null) {
+            // Wait bucket snapshot create finish
+            createFuture.get();
+        }
+
+        final String bucketKey = bucket.bucketKey();
+        final Long bucketId = getBucketIdByBucketKey(bucketKey);
+        Objects.requireNonNull(bucketId);
+
+        CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+        if (isRecover) {
+            // TODO Recover bucket snapshot
+        } else {
+            loadMetaDataFuture.complete(bucket.currentSegmentEntryId + 1);
+        }
+
+        CompletableFuture<Void> future = loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+            if (nextSegmentEntryId > bucket.lastSegmentEntryId) {
+                // TODO Delete bucket snapshot
+                return CompletableFuture.completedFuture(null);
+            }
+
+            return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                    .thenAccept(bucketSnapshotSegments -> {
+                        if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                            return;
+                        }
+
+                        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                        List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                        DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                        this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                lastDelayedIndex.getEntryId(), bucket);
+
+                        for (DelayedIndex index : indexList) {
+                            sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                    index.getEntryId());
+                        }
+
+                        bucket.setCurrentSegmentEntryId(nextSegmentEntryId);
+                    });
+        });
+        return future;
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucket.setStartLedgerId(-1L);
+        lastMutableBucket.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucket.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                if (immutableBuckets.asMapOfRanges().size() >= maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+            }
+        }
+
+        if (ledgerId < lastMutableBucket.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucket.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucket.startLedgerId == -1L) {
+                lastMutableBucket.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucket.setEndLedgerId(ledgerId);
+        }
+
+        // TODO If the bitSet is sparse, this memory cost very high to deduplication and skip read message
+        lastMutableBucket.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {
+        long cutoffTime = getCutoffTime();
+
+        boolean hasMessageAvailable = !getPriorityQueue().isEmpty() && getPriorityQueue().peekN1() <= cutoffTime;
+
+        hasMessageAvailable = hasMessageAvailable
+                || !sharedBucketPriorityQueue.isEmpty() && sharedBucketPriorityQueue.peekN1() <= cutoffTime;
+        if (!hasMessageAvailable) {
+            updateTimer();
+        }
+        return hasMessageAvailable;
+    }
+
+    @Override
+    protected long nextDeliveryTime() {
+        if (getPriorityQueue().isEmpty() && !sharedBucketPriorityQueue.isEmpty()) {
+            return sharedBucketPriorityQueue.peekN1();
+        } else if (sharedBucketPriorityQueue.isEmpty() && !getPriorityQueue().isEmpty()) {
+            return getPriorityQueue().peekN1();
+        }
+        long timestamp = getPriorityQueue().peekN1();
+        long bucketTimestamp = sharedBucketPriorityQueue.peekN1();
+        return Math.min(timestamp, bucketTimestamp);
+    }
+
+    @Override
+    public synchronized long getNumberOfDelayedMessages() {
+        return numberDelayedMessages;
+    }
+
+    @Override
+    public synchronized long getBufferMemoryUsage() {
+        return getPriorityQueue().bytesCapacity() + sharedBucketPriorityQueue.bytesCapacity();
+    }
+
+    @Override
+    @SneakyThrows
+    public synchronized Set<PositionImpl> getScheduledMessages(int maxMessages) {
+        long cutoffTime = getCutoffTime();
+
+        moveScheduledMessageToSharedQueue(cutoffTime);
+
+        Set<PositionImpl> positions = new TreeSet<>();
+        int n = maxMessages;
+
+        while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) {
+            long timestamp = sharedBucketPriorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = sharedBucketPriorityQueue.peekN2();
+            long entryId = sharedBucketPriorityQueue.peekN3();
+            positions.add(new PositionImpl(ledgerId, entryId));
+
+            sharedBucketPriorityQueue.pop();
+            removeIndexBit(ledgerId, entryId);
+
+            Bucket bucket = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
+            if (bucket != null && bucket.active) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Load next snapshot segment, bucket: {}", dispatcher.getName(), bucket);
+                }
+                // All message of current snapshot segment are scheduled, load next snapshot segment
+                asyncLoadNextBucketSnapshotEntry(bucket, false).get();
+            }
+
+            --n;
+            --numberDelayedMessages;
+        }
+
+        if (numberDelayedMessages <= 0) {
+            // Reset to initial state
+            highestDeliveryTimeTracked = 0;
+            messagesHaveFixedDelay = true;
+        }
+
+        updateTimer();
+
+        return positions;
+    }
+
+    @Override
+    @SneakyThrows
+    public synchronized void clear() {
+        super.clear();
+        cleanImmutableBuckets(true);
+        sharedBucketPriorityQueue.clear();
+        resetLastMutableBucketRange();
+        lastMutableBucket.delayedIndexBitMap.clear();
+        snapshotSegmentLastIndexTable.clear();
+        numberDelayedMessages = 0;
+    }
+
+    @Override
+    @SneakyThrows
+    public synchronized void close() {
+        super.close();
+        cleanImmutableBuckets(false);
+        lastMutableBucket.delayedIndexBitMap.clear();
+        sharedBucketPriorityQueue.close();
+    }
+
+    private void cleanImmutableBuckets(boolean delete) {
+        if (immutableBuckets != null) {
+            Iterator<Bucket> iterator = immutableBuckets.asMapOfRanges().values().iterator();
+            while (iterator.hasNext()) {
+                Bucket bucket = iterator.next();
+                if (bucket.delayedIndexBitMap != null) {
+                    bucket.delayedIndexBitMap.clear();
+                }
+                CompletableFuture<Long> snapshotGenerateFuture = bucket.snapshotCreateFuture;
+                if (snapshotGenerateFuture != null) {
+                    if (delete) {
+                        snapshotGenerateFuture.cancel(true);
+                        // TODO delete bucket snapshot
+                    } else {
+                        try {
+                            snapshotGenerateFuture.get();

Review Comment:
   It's better to add timeout.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);

Review Comment:
   because we return `CompletableFuture<Long>`, if we return null it will have a risk to cause NPE.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {

Review Comment:
   Maybe change the `k` to `ledger` can help understand the logic...



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {

Review Comment:
   What happen when `entryId > Integer.MAX_VALUE`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {
+                boolean exist = delayedIndexBitMap.containsKey(k);

Review Comment:
   Because we clear it at line 59, why do we have to check it again?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);

Review Comment:
   If you create an immutable bucket and then we can set some things.  it will break the immutable sematic.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {
+                boolean exist = delayedIndexBitMap.containsKey(k);
+                byte[] bytes = v.toByteArray();
+                BitSet bitSet = BitSet.valueOf(bytes);
+                numberMessages.add(bitSet.cardinality());
+                if (!exist) {
+                    delayedIndexBitMap.put(k, bitSet);
+                } else {
+                    delayedIndexBitMap.get(k).or(bitSet);
+                }
+            });
+        }
+        return numberMessages.longValue();

Review Comment:
   Should we set `numberBucketDelayedMessages` here?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {
+                boolean exist = delayedIndexBitMap.containsKey(k);
+                byte[] bytes = v.toByteArray();
+                BitSet bitSet = BitSet.valueOf(bytes);
+                numberMessages.add(bitSet.cardinality());
+                if (!exist) {
+                    delayedIndexBitMap.put(k, bitSet);
+                } else {
+                    delayedIndexBitMap.get(k).or(bitSet);
+                }
+            });
+        }
+        return numberMessages.longValue();
+    }
+
+    boolean containsMessage(long ledgerId, int entryId) {
+        if (delayedIndexBitMap == null) {
+            return false;
+        }
+
+        BitSet bitSet = delayedIndexBitMap.get(ledgerId);
+        if (bitSet == null) {
+            return false;
+        }
+        return bitSet.get(entryId);
+    }
+
+    void putIndexBit(long ledgerId, long entryId) {
+        if (entryId < Integer.MAX_VALUE) {

Review Comment:
   Why we don't handle the `entryId >= Integer.MAX_VALUE`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {
+                boolean exist = delayedIndexBitMap.containsKey(k);
+                byte[] bytes = v.toByteArray();
+                BitSet bitSet = BitSet.valueOf(bytes);
+                numberMessages.add(bitSet.cardinality());
+                if (!exist) {
+                    delayedIndexBitMap.put(k, bitSet);
+                } else {
+                    delayedIndexBitMap.get(k).or(bitSet);
+                }
+            });
+        }
+        return numberMessages.longValue();
+    }
+
+    boolean containsMessage(long ledgerId, int entryId) {
+        if (delayedIndexBitMap == null) {
+            return false;
+        }
+
+        BitSet bitSet = delayedIndexBitMap.get(ledgerId);
+        if (bitSet == null) {
+            return false;
+        }
+        return bitSet.get(entryId);
+    }
+
+    void putIndexBit(long ledgerId, long entryId) {
+        if (entryId < Integer.MAX_VALUE) {
+            delayedIndexBitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);

Review Comment:
   Should we use `computeIfAbsent`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows

Review Comment:
   Why `@SneakyThrows`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();

Review Comment:
   `this.numberDelayedMessages` ?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(Bucket bucket, boolean isRebuild) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucket: {}", dispatcher.getName(), bucket);
+        }
+        if (bucket == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        final CompletableFuture<Long> createFuture = bucket.snapshotCreateFuture;
+        if (createFuture != null) {
+            // Wait bucket snapshot create finish
+            createFuture.get();

Review Comment:
   We don't suggest using the blocking calls in the async method.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);

Review Comment:
   I see many `checkArgument` methods in the several async methods, it has a risk to throw the exception and we may forget to handle it.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();

Review Comment:
   Why can't we make it to be async?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);

Review Comment:
   `computeIfAbsent`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)

Review Comment:
   Is there any chance it will be empty?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();

Review Comment:
   If `recoverBucketSnapshot` throws an exception, the `lastMutableBucket` will be null.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java:
##########
@@ -274,22 +279,29 @@ public void run(Timeout timeout) throws Exception {
 
     @Override
     public void close() {
-        priorityQueue.close();
         if (timeout != null) {
             timeout.cancel();
         }
+        priorityQueue.close();

Review Comment:
   Why move it?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(Bucket bucket, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucket: {}", dispatcher.getName(), bucket);
+        }
+        if (bucket == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        final CompletableFuture<Long> createFuture = bucket.snapshotCreateFuture;
+        if (createFuture != null) {
+            // Wait bucket snapshot create finish
+            createFuture.get();
+        }
+
+        final String bucketKey = bucket.bucketKey();
+        final Long bucketId = getBucketIdByBucketKey(bucketKey);
+        Objects.requireNonNull(bucketId);
+
+        CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+        if (isRecover) {
+            // TODO Recover bucket snapshot
+        } else {
+            loadMetaDataFuture.complete(bucket.currentSegmentEntryId + 1);
+        }
+
+        CompletableFuture<Void> future = loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+            if (nextSegmentEntryId > bucket.lastSegmentEntryId) {
+                // TODO Delete bucket snapshot
+                return CompletableFuture.completedFuture(null);
+            }
+
+            return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                    .thenAccept(bucketSnapshotSegments -> {
+                        if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                            return;
+                        }
+
+                        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                        List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                        DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                        this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                lastDelayedIndex.getEntryId(), bucket);
+
+                        for (DelayedIndex index : indexList) {
+                            sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                    index.getEntryId());
+                        }
+
+                        bucket.setCurrentSegmentEntryId(nextSegmentEntryId);
+                    });
+        });
+        return future;
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucket.setStartLedgerId(-1L);
+        lastMutableBucket.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucket.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                if (immutableBuckets.asMapOfRanges().size() >= maxNumBuckets) {
+                    // TODO merge bucket snapshot (synchronize operate)
+                }
+
+                asyncCreateBucketSnapshot();
+                resetLastMutableBucketRange();
+            }
+        }
+
+        if (ledgerId < lastMutableBucket.startLedgerId || existBucket) {
+            // If (ledgerId < startLedgerId || existBucket) means that message index belong to previous bucket range,
+            // enter sharedBucketPriorityQueue directly
+            sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId);
+        } else {
+            checkArgument(ledgerId >= lastMutableBucket.endLedgerId);
+
+            getPriorityQueue().add(deliverAt, ledgerId, entryId);
+
+            if (lastMutableBucket.startLedgerId == -1L) {
+                lastMutableBucket.setStartLedgerId(ledgerId);
+            }
+            lastMutableBucket.setEndLedgerId(ledgerId);
+        }
+
+        // TODO If the bitSet is sparse, this memory cost very high to deduplication and skip read message
+        lastMutableBucket.putIndexBit(ledgerId, entryId);
+        numberDelayedMessages++;
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId,
+                    deliverAt - clock.millis());
+        }
+
+        updateTimer();
+
+        checkAndUpdateHighest(deliverAt);
+
+        return true;
+    }
+
+    @Override
+    public synchronized boolean hasMessageAvailable() {
+        long cutoffTime = getCutoffTime();
+
+        boolean hasMessageAvailable = !getPriorityQueue().isEmpty() && getPriorityQueue().peekN1() <= cutoffTime;
+
+        hasMessageAvailable = hasMessageAvailable
+                || !sharedBucketPriorityQueue.isEmpty() && sharedBucketPriorityQueue.peekN1() <= cutoffTime;
+        if (!hasMessageAvailable) {
+            updateTimer();
+        }
+        return hasMessageAvailable;
+    }
+
+    @Override
+    protected long nextDeliveryTime() {
+        if (getPriorityQueue().isEmpty() && !sharedBucketPriorityQueue.isEmpty()) {
+            return sharedBucketPriorityQueue.peekN1();
+        } else if (sharedBucketPriorityQueue.isEmpty() && !getPriorityQueue().isEmpty()) {
+            return getPriorityQueue().peekN1();
+        }
+        long timestamp = getPriorityQueue().peekN1();
+        long bucketTimestamp = sharedBucketPriorityQueue.peekN1();
+        return Math.min(timestamp, bucketTimestamp);
+    }
+
+    @Override
+    public synchronized long getNumberOfDelayedMessages() {
+        return numberDelayedMessages;
+    }
+
+    @Override
+    public synchronized long getBufferMemoryUsage() {
+        return getPriorityQueue().bytesCapacity() + sharedBucketPriorityQueue.bytesCapacity();
+    }
+
+    @Override
+    @SneakyThrows
+    public synchronized Set<PositionImpl> getScheduledMessages(int maxMessages) {
+        long cutoffTime = getCutoffTime();
+
+        moveScheduledMessageToSharedQueue(cutoffTime);
+
+        Set<PositionImpl> positions = new TreeSet<>();
+        int n = maxMessages;
+
+        while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) {
+            long timestamp = sharedBucketPriorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = sharedBucketPriorityQueue.peekN2();
+            long entryId = sharedBucketPriorityQueue.peekN3();
+            positions.add(new PositionImpl(ledgerId, entryId));
+
+            sharedBucketPriorityQueue.pop();
+            removeIndexBit(ledgerId, entryId);
+
+            Bucket bucket = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
+            if (bucket != null && bucket.active) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Load next snapshot segment, bucket: {}", dispatcher.getName(), bucket);
+                }
+                // All message of current snapshot segment are scheduled, load next snapshot segment
+                asyncLoadNextBucketSnapshotEntry(bucket, false).get();

Review Comment:
   It better to make it async



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = 0L;
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(Bucket bucket, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucket: {}", dispatcher.getName(), bucket);
+        }
+        if (bucket == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        final CompletableFuture<Long> createFuture = bucket.snapshotCreateFuture;
+        if (createFuture != null) {
+            // Wait bucket snapshot create finish
+            createFuture.get();
+        }
+
+        final String bucketKey = bucket.bucketKey();
+        final Long bucketId = getBucketIdByBucketKey(bucketKey);
+        Objects.requireNonNull(bucketId);
+
+        CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+        if (isRecover) {
+            // TODO Recover bucket snapshot
+        } else {
+            loadMetaDataFuture.complete(bucket.currentSegmentEntryId + 1);
+        }
+
+        CompletableFuture<Void> future = loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {

Review Comment:
   we can return directly.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);
+            }
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, BitSet>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, BitSet> entry = iterator.next();
+                    ByteString byteString = ByteString.copyFrom(entry.getValue().toByteArray());
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), byteString);
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        Bucket bucket = this.createImmutableBucket(startLedgerId, endLedgerId);
+        bucket.setCurrentSegmentEntryId(1);
+        bucket.setNumberBucketDelayedMessages(numMessages);
+        bucket.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucket);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucket: {}", dispatcher.getName(), bucket);
+        }
+
+        String bucketKey = bucket.bucketKey();
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketKey,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucket.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucket.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    @SneakyThrows
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(Bucket bucket, boolean isRebuild) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucket: {}", dispatcher.getName(), bucket);
+        }
+        if (bucket == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        final CompletableFuture<Long> createFuture = bucket.snapshotCreateFuture;
+        if (createFuture != null) {
+            // Wait bucket snapshot create finish
+            createFuture.get();
+        }
+
+        final String bucketKey = bucket.bucketKey();
+        final Long bucketId = getBucketIdByBucketKey(bucketKey);
+        Objects.requireNonNull(bucketId);
+
+        CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+        if (isRebuild) {
+            final long cutoffTime = getCutoffTime();
+            // Load Metadata of bucket snapshot
+            bucketSnapshotStorage.getBucketSnapshotMetadata(bucketId).thenAccept(snapshotMetadata -> {
+                List<SnapshotSegmentMetadata> metadataList = snapshotMetadata.getMetadataListList();
+
+                // Skip all already reach schedule time snapshot segments
+                int nextSnapshotEntryIndex = 0;
+                while (nextSnapshotEntryIndex < metadataList.size()
+                        && metadataList.get(nextSnapshotEntryIndex).getMaxScheduleTimestamp() <= cutoffTime) {
+                    nextSnapshotEntryIndex++;
+                }
+
+                final int lastSegmentEntryId = metadataList.size();
+
+                long numberMessages = bucket.covertDelayIndexMapAndCount(nextSnapshotEntryIndex, metadataList);

Review Comment:
   Move to another PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {

Review Comment:
   Move to another PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java:
##########
@@ -274,22 +279,29 @@ public void run(Timeout timeout) throws Exception {
 
     @Override
     public void close() {
-        priorityQueue.close();
         if (timeout != null) {
             timeout.cancel();
         }
+        priorityQueue.close();

Review Comment:
   Ok, I push a new pr to fix it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";

Review Comment:
   Make sense.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();

Review Comment:
   Move to another PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,628 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final Bucket lastMutableBucket;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, Bucket> immutableBuckets;
+
+    private final Table<Long, Long, Bucket> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        numberDelayedMessages = recoverBucketSnapshot();
+
+        this.lastMutableBucket = new Bucket(-1L, -1L, new HashMap<>());
+    }
+
+    @SneakyThrows
+    private long recoverBucketSnapshot() {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        this.cursor.getCursorProperties().keySet().forEach(key -> {
+            if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
+                String[] keys = key.split(DELIMITER);
+                checkArgument(keys.length == 3);
+                Bucket bucket = createImmutableBucket(Long.parseLong(keys[1]), Long.parseLong(keys[2]));
+                completableFutures.add(asyncLoadNextBucketSnapshotEntry(bucket, true));
+            }
+        });
+
+        if (completableFutures.isEmpty()) {
+            return 0;
+        }
+
+        FutureUtil.waitForAll(completableFutures).get();
+
+        MutableLong numberDelayedMessages = new MutableLong(0);
+        immutableBuckets.asMapOfRanges().values().forEach(bucket -> {
+            numberDelayedMessages.add(bucket.numberBucketDelayedMessages);
+        });
+
+        log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}",
+                dispatcher.getName(), immutableBuckets.asMapOfRanges().size(), numberDelayedMessages.getValue());
+
+        return numberDelayedMessages.getValue();
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<Bucket> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        Range<Long> span = immutableBuckets.span();
+        if (!span.contains(ledgerId)) {
+            return Optional.empty();
+        }
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private Long getBucketIdByBucketKey(String bucketKey) {
+        String bucketIdStr = cursor.getCursorProperties().get(bucketKey);
+        if (StringUtils.isBlank(bucketIdStr)) {
+            return null;
+        }
+        return Long.valueOf(bucketIdStr);
+    }
+
+    private Bucket createImmutableBucket(long startLedgerId, long endLedgerId) {
+        Bucket newBucket = new Bucket(startLedgerId, endLedgerId, new HashMap<>());
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), newBucket);
+        return newBucket;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            final String bucketKey, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+        Long bucketId = getBucketIdByBucketKey(bucketKey);
+        checkArgument(bucketId == null);
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> putBucketKeyId(bucketKey, newBucketId));
+    }
+
+    private CompletableFuture<Long> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class).thenApply(__ -> bucketId);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucket.startLedgerId;
+        final long endLedgerId = lastMutableBucket.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, BitSet> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            if (entryId <= Integer.MAX_VALUE) {
+                bitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);

Review Comment:
   Yes, you are right.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {
+                boolean exist = delayedIndexBitMap.containsKey(k);

Review Comment:
   Move to another PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/Bucket.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX;
+import static org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTracker.DELIMITER;
+import com.google.protobuf.ByteString;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+
+@Data
+@AllArgsConstructor
+public class Bucket {
+
+    long startLedgerId;
+    long endLedgerId;
+
+    Map<Long, BitSet> delayedIndexBitMap;
+
+    long numberBucketDelayedMessages;
+
+    int lastSegmentEntryId;
+
+    int currentSegmentEntryId;
+
+    long snapshotLength;
+
+    boolean active;
+
+    volatile CompletableFuture<Long> snapshotCreateFuture;
+
+    Bucket(long startLedgerId, long endLedgerId, Map<Long, BitSet> delayedIndexBitMap) {
+        this(startLedgerId, endLedgerId, delayedIndexBitMap, -1, -1, 0, 0, true, null);
+    }
+
+    long covertDelayIndexMapAndCount(int startSnapshotIndex, List<SnapshotSegmentMetadata> segmentMetadata) {
+        delayedIndexBitMap.clear();
+        MutableLong numberMessages = new MutableLong(0);
+        for (int i = startSnapshotIndex; i < segmentMetadata.size(); i++) {
+            Map<Long, ByteString> bitByteStringMap = segmentMetadata.get(i).getDelayedIndexBitMapMap();
+            bitByteStringMap.forEach((k, v) -> {
+                boolean exist = delayedIndexBitMap.containsKey(k);
+                byte[] bytes = v.toByteArray();
+                BitSet bitSet = BitSet.valueOf(bytes);
+                numberMessages.add(bitSet.cardinality());
+                if (!exist) {
+                    delayedIndexBitMap.put(k, bitSet);
+                } else {
+                    delayedIndexBitMap.get(k).or(bitSet);
+                }
+            });
+        }
+        return numberMessages.longValue();
+    }
+
+    boolean containsMessage(long ledgerId, int entryId) {
+        if (delayedIndexBitMap == null) {
+            return false;
+        }
+
+        BitSet bitSet = delayedIndexBitMap.get(ledgerId);
+        if (bitSet == null) {
+            return false;
+        }
+        return bitSet.get(entryId);
+    }
+
+    void putIndexBit(long ledgerId, long entryId) {
+        if (entryId < Integer.MAX_VALUE) {
+            delayedIndexBitMap.compute(ledgerId, (k, v) -> new BitSet()).set((int) entryId);

Review Comment:
   Ok.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();

Review Comment:
   I check all use cases, they will be synchronous by `synchronized`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] gaoran10 commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),

Review Comment:
   I'm not sure we can always try to modify the property when encountering `ManagedLedgerException.BadVersionException`, in common, it's ok, I'm afraid it's become a dead cycle when the data version of the cursor lower than the metadata store, maybe this is another problem.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))

Review Comment:
   Yes, the result of future be unused.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))
+                        .thenApply(__ -> null);
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {
+                asyncCreateBucketSnapshot();

Review Comment:
   Good idea.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] gaoran10 commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    /**
+     * Asynchronous load next bucket snapshot entry.
+     * @param bucketState bucket state
+     * @param isRecover whether used to recover bucket snapshot
+     * @return CompletableFuture
+     */
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Void> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(null))

Review Comment:
   Can the future be an empty object? I'm not sure about the comment `Wait bucket snapshot create finish`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [pulsar] gaoran10 commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),

Review Comment:
   Yes, I think so.



-- 
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] coderzc commented on a diff in pull request #17611: [feat][broker][PIP-195] Implement delayed message index bucket snapshot (create/load) - part2

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Long> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(-1L));
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {

Review Comment:
   Oh, seems to be this condition should be `&& ! existBucket `, I fix it.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTracker.java:
##########
@@ -0,0 +1,565 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.delayed;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeRangeMap;
+import com.google.protobuf.ByteString;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.nio.ByteBuffer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.concurrent.ThreadSafe;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;
+import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata;
+import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.roaringbitmap.RoaringBitmap;
+
+@Slf4j
+@ThreadSafe
+public class BucketDelayedDeliveryTracker extends InMemoryDelayedDeliveryTracker {
+
+    protected static final int AsyncOperationTimeoutSeconds = 30;
+
+    public static final String DELAYED_BUCKET_KEY_PREFIX = "#pulsar.internal.delayed.bucket";
+
+    public static final String DELIMITER = "_";
+
+    private final long minIndexCountPerBucket;
+
+    private final long timeStepPerBucketSnapshotSegment;
+
+    private final int maxNumBuckets;
+
+    private final ManagedCursor cursor;
+
+    public final BucketSnapshotStorage bucketSnapshotStorage;
+
+    private long numberDelayedMessages;
+
+    private final BucketState lastMutableBucketState;
+
+    private final TripleLongPriorityQueue sharedBucketPriorityQueue;
+
+    private final RangeMap<Long, BucketState> immutableBuckets;
+
+    private final Table<Long, Long, BucketState> snapshotSegmentLastIndexTable;
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead,
+                bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegment, maxNumBuckets);
+    }
+
+    BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
+                                 Timer timer, long tickTimeMillis, Clock clock,
+                                 boolean isDelayedDeliveryDeliverAtTimeStrict,
+                                 long fixedDelayDetectionLookahead,
+                                 BucketSnapshotStorage bucketSnapshotStorage,
+                                 long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegment,
+                                 int maxNumBuckets) {
+        super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict,
+                fixedDelayDetectionLookahead);
+        this.minIndexCountPerBucket = minIndexCountPerBucket;
+        this.timeStepPerBucketSnapshotSegment = timeStepPerBucketSnapshotSegment;
+        this.maxNumBuckets = maxNumBuckets;
+        this.cursor = dispatcher.getCursor();
+        this.sharedBucketPriorityQueue = new TripleLongPriorityQueue();
+        this.immutableBuckets = TreeRangeMap.create();
+        this.snapshotSegmentLastIndexTable = HashBasedTable.create();
+
+        this.bucketSnapshotStorage = bucketSnapshotStorage;
+
+        this.numberDelayedMessages = 0L;
+
+        this.lastMutableBucketState = new BucketState(-1L, -1L);
+    }
+
+    private void moveScheduledMessageToSharedQueue(long cutoffTime) {
+        TripleLongPriorityQueue priorityQueue = getPriorityQueue();
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (timestamp > cutoffTime) {
+                break;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+            sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+
+            priorityQueue.pop();
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        synchronized (this) {
+            if (timeout == null || timeout.isCancelled()) {
+                return;
+            }
+            moveScheduledMessageToSharedQueue(getCutoffTime());
+        }
+        super.run(timeout);
+    }
+
+    private Optional<BucketState> findBucket(long ledgerId) {
+        if (immutableBuckets.asMapOfRanges().isEmpty()) {
+            return Optional.empty();
+        }
+
+        return Optional.ofNullable(immutableBuckets.get(ledgerId));
+    }
+
+    private long getBucketId(BucketState bucketState) {
+        Optional<Long> bucketIdOptional = bucketState.getBucketId();
+        if (bucketIdOptional.isPresent()) {
+            return bucketIdOptional.get();
+        }
+
+        String bucketIdStr = cursor.getCursorProperties().get(bucketState.bucketKey());
+        long bucketId = Long.parseLong(bucketIdStr);
+        bucketState.setBucketId(bucketId);
+        return bucketId;
+    }
+
+    private BucketState createBucket(long startLedgerId, long endLedgerId) {
+        BucketState bucketState = new BucketState(startLedgerId, endLedgerId);
+        immutableBuckets.put(Range.closed(startLedgerId, endLedgerId), bucketState);
+        return bucketState;
+    }
+
+    private CompletableFuture<Long> asyncSaveBucketSnapshot(
+            BucketState bucketState, SnapshotMetadata snapshotMetadata,
+            List<SnapshotSegment> bucketSnapshotSegments) {
+
+        return bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments)
+                .thenCompose(newBucketId -> {
+                    bucketState.setBucketId(newBucketId);
+                    String bucketKey = bucketState.bucketKey();
+                    return putBucketKeyId(bucketKey, newBucketId).exceptionally(ex -> {
+                        log.warn("Failed to record bucketId to cursor property, bucketKey: {}", bucketKey);
+                        return null;
+                    }).thenApply(__ -> newBucketId);
+                });
+    }
+
+    private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId) {
+        Objects.requireNonNull(bucketId);
+        return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
+                ManagedLedgerException.BadVersionException.class);
+    }
+
+    private CompletableFuture<Long> asyncCreateBucketSnapshot() {
+        TripleLongPriorityQueue priorityQueue = super.getPriorityQueue();
+        if (priorityQueue.isEmpty()) {
+            return CompletableFuture.completedFuture(-1L);
+        }
+        long numMessages = 0;
+
+        final long startLedgerId = lastMutableBucketState.startLedgerId;
+        final long endLedgerId = lastMutableBucketState.endLedgerId;
+
+        List<SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();
+        List<SnapshotSegmentMetadata> segmentMetadataList = new ArrayList<>();
+        Map<Long, RoaringBitmap> bitMap = new HashMap<>();
+        SnapshotSegment.Builder snapshotSegmentBuilder = SnapshotSegment.newBuilder();
+        SnapshotSegmentMetadata.Builder segmentMetadataBuilder = SnapshotSegmentMetadata.newBuilder();
+
+        long currentTimestampUpperLimit = 0;
+        while (!priorityQueue.isEmpty()) {
+            long timestamp = priorityQueue.peekN1();
+            if (currentTimestampUpperLimit == 0) {
+                currentTimestampUpperLimit = timestamp + timeStepPerBucketSnapshotSegment - 1;
+            }
+
+            long ledgerId = priorityQueue.peekN2();
+            long entryId = priorityQueue.peekN3();
+
+            checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
+
+            // Move first segment of bucket snapshot to sharedBucketPriorityQueue
+            if (segmentMetadataList.size() == 0) {
+                sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
+            }
+
+            priorityQueue.pop();
+            numMessages++;
+
+            DelayedIndex delayedIndex = DelayedIndex.newBuilder()
+                    .setTimestamp(timestamp)
+                    .setLedgerId(ledgerId)
+                    .setEntryId(entryId).build();
+
+            bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1);
+
+            snapshotSegmentBuilder.addIndexes(delayedIndex);
+
+            if (priorityQueue.isEmpty() || priorityQueue.peekN1() > currentTimestampUpperLimit) {
+                segmentMetadataBuilder.setMaxScheduleTimestamp(timestamp);
+                currentTimestampUpperLimit = 0;
+
+                Iterator<Map.Entry<Long, RoaringBitmap>> iterator = bitMap.entrySet().iterator();
+                while (iterator.hasNext()) {
+                    Map.Entry<Long, RoaringBitmap> entry = iterator.next();
+                    byte[] array = new byte[entry.getValue().serializedSizeInBytes()];
+                    entry.getValue().serialize(ByteBuffer.wrap(array));
+                    segmentMetadataBuilder.putDelayedIndexBitMap(entry.getKey(), ByteString.copyFrom(array));
+                    iterator.remove();
+                }
+
+                segmentMetadataList.add(segmentMetadataBuilder.build());
+                segmentMetadataBuilder.clear();
+
+                bucketSnapshotSegments.add(snapshotSegmentBuilder.build());
+                snapshotSegmentBuilder.clear();
+            }
+        }
+
+        SnapshotMetadata bucketSnapshotMetadata = SnapshotMetadata.newBuilder()
+                .addAllMetadataList(segmentMetadataList)
+                .build();
+
+        final int lastSegmentEntryId = segmentMetadataList.size();
+
+        BucketState bucketState = this.createBucket(startLedgerId, endLedgerId);
+        bucketState.setCurrentSegmentEntryId(1);
+        bucketState.setNumberBucketDelayedMessages(numMessages);
+        bucketState.setLastSegmentEntryId(lastSegmentEntryId);
+
+        // Add the first snapshot segment last message to snapshotSegmentLastMessageTable
+        checkArgument(!bucketSnapshotSegments.isEmpty());
+        SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+        DelayedIndex delayedIndex = snapshotSegment.getIndexes(snapshotSegment.getIndexesCount() - 1);
+        snapshotSegmentLastIndexTable.put(delayedIndex.getLedgerId(), delayedIndex.getEntryId(), bucketState);
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Create bucket snapshot, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+
+        CompletableFuture<Long> future = asyncSaveBucketSnapshot(bucketState,
+                bucketSnapshotMetadata, bucketSnapshotSegments);
+        bucketState.setSnapshotCreateFuture(future);
+        future.whenComplete((__, ex) -> {
+            if (ex == null) {
+                bucketState.setSnapshotCreateFuture(null);
+            } else {
+                //TODO Record create snapshot failed
+                log.error("Failed to create snapshot: ", ex);
+            }
+        });
+
+        return future;
+    }
+
+
+    private CompletableFuture<Void> asyncLoadNextBucketSnapshotEntry(BucketState bucketState, boolean isRecover) {
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] Load next bucket snapshot data, bucketState: {}", dispatcher.getName(), bucketState);
+        }
+        if (bucketState == null) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        // Wait bucket snapshot create finish
+        CompletableFuture<Long> snapshotCreateFuture =
+                bucketState.getSnapshotCreateFuture().orElseGet(() -> CompletableFuture.completedFuture(-1L));
+
+        return snapshotCreateFuture.thenCompose(__ -> {
+            final long bucketId = getBucketId(bucketState);
+            CompletableFuture<Integer> loadMetaDataFuture = new CompletableFuture<>();
+            if (isRecover) {
+                // TODO Recover bucket snapshot
+            } else {
+                loadMetaDataFuture.complete(bucketState.currentSegmentEntryId + 1);
+            }
+
+            return loadMetaDataFuture.thenCompose(nextSegmentEntryId -> {
+                if (nextSegmentEntryId > bucketState.lastSegmentEntryId) {
+                    // TODO Delete bucket snapshot
+                    return CompletableFuture.completedFuture(null);
+                }
+
+                return bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, nextSegmentEntryId, nextSegmentEntryId)
+                        .thenAccept(bucketSnapshotSegments -> {
+                            if (CollectionUtils.isEmpty(bucketSnapshotSegments)) {
+                                return;
+                            }
+
+                            SnapshotSegment snapshotSegment = bucketSnapshotSegments.get(0);
+                            List<DelayedIndex> indexList = snapshotSegment.getIndexesList();
+                            DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
+
+                            this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
+                                    lastDelayedIndex.getEntryId(), bucketState);
+
+                            for (DelayedIndex index : indexList) {
+                                sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
+                                        index.getEntryId());
+                            }
+
+                            bucketState.setCurrentSegmentEntryId(nextSegmentEntryId);
+                        });
+            });
+        });
+    }
+
+    private void resetLastMutableBucketRange() {
+        lastMutableBucketState.setStartLedgerId(-1L);
+        lastMutableBucketState.setEndLedgerId(-1L);
+    }
+
+    @Override
+    public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) {
+        if (containsMessage(ledgerId, entryId)) {
+            messagesHaveFixedDelay = false;
+            return true;
+        }
+
+        if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
+            messagesHaveFixedDelay = false;
+            return false;
+        }
+
+        boolean existBucket = findBucket(ledgerId).isPresent();
+
+        // Create bucket snapshot
+        if (ledgerId > lastMutableBucketState.endLedgerId && !getPriorityQueue().isEmpty()) {
+            if (getPriorityQueue().size() >= minIndexCountPerBucket || existBucket) {

Review Comment:
   Oh, seems to be this condition should be `&& !existBucket `, I fix it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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