diff --git a/microbench/src/main/java/org/apache/bookkeeper/mledger/impl/MockManagedCursor.java b/microbench/src/main/java/org/apache/bookkeeper/mledger/impl/MockManagedCursor.java index af602a8a9fe50..927b4a1c9299e 100644 --- a/microbench/src/main/java/org/apache/bookkeeper/mledger/impl/MockManagedCursor.java +++ b/microbench/src/main/java/org/apache/bookkeeper/mledger/impl/MockManagedCursor.java @@ -36,7 +36,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; -class MockManagedCursor implements ManagedCursor { +public class MockManagedCursor implements ManagedCursor { ActiveManagedCursorContainer container; Position markDeletePosition; Position readPosition; @@ -58,7 +58,8 @@ class MockManagedCursor implements ManagedCursor { this.durable = durable; } - static MockManagedCursor createCursor(ActiveManagedCursorContainer container, String name, Position position) { + public static MockManagedCursor createCursor(ActiveManagedCursorContainer container, String name, + Position position) { return new MockManagedCursor(container, name, position, position, false, true); } diff --git a/microbench/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerBenchmark.java b/microbench/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerBenchmark.java new file mode 100644 index 0000000000000..f0faa1f6d3ab6 --- /dev/null +++ b/microbench/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerBenchmark.java @@ -0,0 +1,310 @@ +/* + * 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.bucket; + +import io.netty.util.HashedWheelTimer; +import io.netty.util.Timer; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.time.Clock; +import java.util.NavigableSet; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.ActiveManagedCursorContainerImpl; +import org.apache.bookkeeper.mledger.impl.MockManagedCursor; +import org.apache.pulsar.broker.delayed.DelayedDeliveryTracker; +import org.apache.pulsar.broker.delayed.NoopDelayedDeliveryContext; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +/** + * Enhanced JMH Benchmarks for BucketDelayedDeliveryTracker with ReentrantReadWriteLock. + * This benchmark tests the performance improvements made by transitioning from + * StampedLock to ReentrantReadWriteLock for fine-grained concurrency control. + *

+ * Run with: mvn exec:java -Dexec.mainClass="org.openjdk.jmh.Main" + * -Dexec.args="BucketDelayedDeliveryTrackerBenchmark" + */ +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.SECONDS) +@State(Scope.Benchmark) +@Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) +@Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) +@Fork(1) +public class BucketDelayedDeliveryTrackerBenchmark { + + @Param({"90_10", "80_20", "70_30", "50_50"}) + public String readWriteRatio; + + @Param({"1000", "5000", "8000"}) + public int initialMessages; + + private BucketDelayedDeliveryTracker tracker; + private Timer timer; + private MockBucketSnapshotStorage storage; + private NoopDelayedDeliveryContext context; + private AtomicLong messageIdGenerator; + /** + * Maximum number of additional unique (ledgerId, entryId) positions to + * introduce per trial on top of {@link #initialMessages}. This allows + * controlling the memory footprint of the benchmark while still applying + * sustained write pressure to the tracker. + * + *

Use {@code -p maxAdditionalUniqueMessages=...} on the JMH command line + * to tune the load. The default value is conservative for local runs.

+ */ + @Param({"1000000"}) + public long maxAdditionalUniqueMessages; + /** + * Upper bound on the absolute message id that will be used to derive + * (ledgerId, entryId) positions during a single trial. + */ + private long maxUniqueMessageId; + /** + * In real Pulsar usage, {@link DelayedDeliveryTracker#addMessage(long, long, long)} is invoked + * by a single dispatcher thread and messages arrive in order of (ledgerId, entryId). + *

+ * To reflect this invariant in the benchmark, all write operations that end up calling + * {@code tracker.addMessage(...)} are serialized via this mutex so that the tracker only + * ever observes a single writer with monotonically increasing ids, even when JMH runs the + * benchmark method with multiple threads. + */ + private final Object writeMutex = new Object(); + + @Setup(Level.Trial) + public void setup() throws Exception { + setupMockComponents(); + createTracker(); + preloadMessages(); + messageIdGenerator = new AtomicLong(initialMessages + 1); + // Allow a bounded number of additional unique messages per trial to avoid + // unbounded memory growth while still stressing the indexing logic. + maxUniqueMessageId = initialMessages + maxAdditionalUniqueMessages; + } + + @TearDown(Level.Trial) + public void tearDown() throws Exception { + if (tracker != null) { + tracker.close(); + } + if (timer != null) { + timer.stop(); + } + } + + private void setupMockComponents() throws Exception { + timer = new HashedWheelTimer(new DefaultThreadFactory("test-delayed-delivery"), 100, TimeUnit.MILLISECONDS); + storage = new MockBucketSnapshotStorage(); + + ActiveManagedCursorContainerImpl container = new ActiveManagedCursorContainerImpl(); + MockManagedCursor cursor = MockManagedCursor.createCursor(container, "test-cursor", + PositionFactory.create(0, 0)); + // Use the same " / " naming pattern as real dispatchers, + // so that Bucket.asyncSaveBucketSnapshot can correctly derive topicName. + String dispatcherName = "persistent://public/default/jmh-topic / " + cursor.getName(); + context = new NoopDelayedDeliveryContext(dispatcherName, cursor); + } + + private void createTracker() throws Exception { + tracker = new BucketDelayedDeliveryTracker( + context, timer, 1000, Clock.systemUTC(), true, storage, + 20, 1000, 100, 50 + ); + } + + private void preloadMessages() { + // Preload messages to create realistic test conditions + long baseTime = System.currentTimeMillis() + 10000; // Future delivery + for (int i = 1; i <= initialMessages; i++) { + tracker.addMessage(i, i, baseTime + i * 1000); + } + } + + // ============================================================================= + // READ-WRITE RATIO BENCHMARKS + // ============================================================================= + + @Benchmark + public boolean benchmarkMixedOperations() { + String[] parts = readWriteRatio.split("_"); + int readPercentage = Integer.parseInt(parts[0]); + + if (ThreadLocalRandom.current().nextInt(100) < readPercentage) { + // Read operations + return performReadOperation(); + } else { + // Write operations + return performWriteOperation(); + } + } + + /** + * Serialize calls to {@link BucketDelayedDeliveryTracker#addMessage(long, long, long)} and + * ensure (ledgerId, entryId) are generated in a strictly increasing sequence, matching the + * real dispatcher single-threaded behaviour. + */ + private boolean addMessageSequential(long deliverAt, int entryIdModulo) { + synchronized (writeMutex) { + long id = messageIdGenerator.getAndIncrement(); + // Limit the number of distinct positions that are introduced into the tracker + // to keep memory usage bounded. Once the upper bound is reached, we re-use + // the last position id so that subsequent calls behave like updates to + // existing messages and are short-circuited by containsMessage checks. + long boundedId = Math.min(id, maxUniqueMessageId); + long ledgerId = boundedId; + long entryId = boundedId % entryIdModulo; + return tracker.addMessage(ledgerId, entryId, deliverAt); + } + } + + private boolean performReadOperation() { + int operation = ThreadLocalRandom.current().nextInt(3); + switch (operation) { + case 0: + // containsMessage + long ledgerId = ThreadLocalRandom.current().nextLong(1, initialMessages + 100); + long entryId = ThreadLocalRandom.current().nextLong(1, 1000); + return tracker.containsMessage(ledgerId, entryId); + case 1: + // nextDeliveryTime + try { + tracker.nextDeliveryTime(); + return true; + } catch (Exception e) { + return false; + } + case 2: + // getNumberOfDelayedMessages + long count = tracker.getNumberOfDelayedMessages(); + return count >= 0; + default: + return false; + } + } + + private boolean performWriteOperation() { + long deliverAt = System.currentTimeMillis() + ThreadLocalRandom.current().nextLong(5000, 30000); + return addMessageSequential(deliverAt, 1000); + } + + // ============================================================================= + // SPECIFIC OPERATION BENCHMARKS + // ============================================================================= + + @Benchmark + @Threads(8) + public boolean benchmarkConcurrentContainsMessage() { + long ledgerId = ThreadLocalRandom.current().nextLong(1, initialMessages + 100); + long entryId = ThreadLocalRandom.current().nextLong(1, 1000); + return tracker.containsMessage(ledgerId, entryId); + } + + @Benchmark + @Threads(4) + public boolean benchmarkConcurrentAddMessage() { + long deliverAt = System.currentTimeMillis() + ThreadLocalRandom.current().nextLong(10000, 60000); + return addMessageSequential(deliverAt, 1000); + } + + @Benchmark + @Threads(2) + public NavigableSet benchmarkConcurrentGetScheduledMessages() { + // Create some messages ready for delivery + long currentTime = System.currentTimeMillis(); + for (int i = 0; i < 5; i++) { + addMessageSequential(currentTime - 1000, 100); + } + return tracker.getScheduledMessages(10); + } + + @Benchmark + @Threads(16) + public long benchmarkConcurrentNextDeliveryTime() { + try { + return tracker.nextDeliveryTime(); + } catch (Exception e) { + return -1; + } + } + + @Benchmark + @Threads(1) + public long benchmarkGetNumberOfDelayedMessages() { + return tracker.getNumberOfDelayedMessages(); + } + + // ============================================================================= + // HIGH CONTENTION SCENARIOS + // ============================================================================= + + @Benchmark + @Threads(32) + public boolean benchmarkHighContentionMixedOperations() { + return benchmarkMixedOperations(); + } + + @Benchmark + @Threads(16) + public boolean benchmarkContentionReads() { + return performReadOperation(); + } + + @Benchmark + @Threads(8) + public boolean benchmarkContentionWrites() { + return performWriteOperation(); + } + + // ============================================================================= + // THROUGHPUT BENCHMARKS + // ============================================================================= + + @Benchmark + @Threads(1) + public boolean benchmarkSingleThreadedThroughput() { + return benchmarkMixedOperations(); + } + + @Benchmark + @Threads(4) + public boolean benchmarkMediumConcurrencyThroughput() { + return benchmarkMixedOperations(); + } + + @Benchmark + @Threads(8) + public boolean benchmarkHighConcurrencyThroughput() { + return benchmarkMixedOperations(); + } + +} diff --git a/microbench/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerSimpleBenchmark.java b/microbench/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerSimpleBenchmark.java deleted file mode 100644 index 985e714d54d1d..0000000000000 --- a/microbench/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerSimpleBenchmark.java +++ /dev/null @@ -1,408 +0,0 @@ -/* - * 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.bucket; - -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.StampedLock; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Level; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.TearDown; -import org.openjdk.jmh.annotations.Threads; -import org.openjdk.jmh.annotations.Warmup; - -/** - * Simplified JMH Benchmarks for BucketDelayedDeliveryTracker thread safety improvements. - * This benchmark focuses on the core StampedLock optimistic read performance without - * complex dependencies on the full BucketDelayedDeliveryTracker implementation. - * Run with: mvn exec:java -Dexec.mainClass="org.openjdk.jmh.Main" - * -Dexec.args="BucketDelayedDeliveryTrackerSimpleBenchmark" - */ -@BenchmarkMode(Mode.Throughput) -@OutputTimeUnit(TimeUnit.SECONDS) -@State(Scope.Benchmark) -@Warmup(iterations = 5, time = 1) -@Measurement(iterations = 5, time = 1) -@Fork(1) -public class BucketDelayedDeliveryTrackerSimpleBenchmark { - - @Param({"1", "2", "4", "8", "16"}) - public int threadCount; - - private StampedLock stampedLock; - private boolean testData = true; - private volatile long counter = 0; - - @Setup(Level.Trial) - public void setup() throws Exception { - stampedLock = new StampedLock(); - } - - @TearDown(Level.Trial) - public void tearDown() throws Exception { - // Cleanup if needed - } - - // ============================================================================= - // STAMPED LOCK OPTIMISTIC READ BENCHMARKS - // ============================================================================= - - @Benchmark - @Threads(1) - public boolean benchmarkOptimisticReadSingleThreaded() { - // Simulate optimistic read like in containsMessage() - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; // Simulate reading shared data - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } - - @Benchmark - @Threads(2) - public boolean benchmarkOptimisticReadMultiThreaded() { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } - - @Benchmark - @Threads(8) - public boolean benchmarkOptimisticReadHighConcurrency() { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } - - @Benchmark - @Threads(16) - public boolean benchmarkOptimisticReadExtremeConcurrency() { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } - - // ============================================================================= - // READ:WRITE RATIO BENCHMARKS (as requested) - // ============================================================================= - - @Benchmark - @Threads(4) - public boolean benchmarkReadWrite10_90() { - // 10:90 read:write ratio simulation - if (ThreadLocalRandom.current().nextInt(100) < 10) { - // Read operation - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } else { - // Write operation - long stamp = stampedLock.writeLock(); - try { - testData = !testData; - counter++; - return testData; - } finally { - stampedLock.unlockWrite(stamp); - } - } - } - - @Benchmark - @Threads(4) - public boolean benchmarkReadWrite20_80() { - // 20:80 read:write ratio - if (ThreadLocalRandom.current().nextInt(100) < 20) { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } else { - long stamp = stampedLock.writeLock(); - try { - testData = !testData; - counter++; - return testData; - } finally { - stampedLock.unlockWrite(stamp); - } - } - } - - @Benchmark - @Threads(4) - public boolean benchmarkReadWrite40_60() { - // 40:60 read:write ratio - if (ThreadLocalRandom.current().nextInt(100) < 40) { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } else { - long stamp = stampedLock.writeLock(); - try { - testData = !testData; - counter++; - return testData; - } finally { - stampedLock.unlockWrite(stamp); - } - } - } - - @Benchmark - @Threads(4) - public boolean benchmarkReadWrite50_50() { - // 50:50 read:write ratio - if (ThreadLocalRandom.current().nextInt(100) < 50) { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } else { - long stamp = stampedLock.writeLock(); - try { - testData = !testData; - counter++; - return testData; - } finally { - stampedLock.unlockWrite(stamp); - } - } - } - - @Benchmark - @Threads(4) - public boolean benchmarkReadWrite60_40() { - // 60:40 read:write ratio - if (ThreadLocalRandom.current().nextInt(100) < 60) { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } else { - long stamp = stampedLock.writeLock(); - try { - testData = !testData; - counter++; - return testData; - } finally { - stampedLock.unlockWrite(stamp); - } - } - } - - @Benchmark - @Threads(4) - public boolean benchmarkReadWrite80_20() { - // 80:20 read:write ratio - if (ThreadLocalRandom.current().nextInt(100) < 80) { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } else { - long stamp = stampedLock.writeLock(); - try { - testData = !testData; - counter++; - return testData; - } finally { - stampedLock.unlockWrite(stamp); - } - } - } - - @Benchmark - @Threads(4) - public boolean benchmarkReadWrite90_10() { - // 90:10 read:write ratio - most realistic for production - if (ThreadLocalRandom.current().nextInt(100) < 90) { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } else { - long stamp = stampedLock.writeLock(); - try { - testData = !testData; - counter++; - return testData; - } finally { - stampedLock.unlockWrite(stamp); - } - } - } - - // ============================================================================= - // HIGH CONCURRENCY SCENARIOS - // ============================================================================= - - @Benchmark - @Threads(8) - public boolean benchmarkReadWrite90_10_HighConcurrency() { - // 90:10 read:write ratio with high concurrency - if (ThreadLocalRandom.current().nextInt(100) < 90) { - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } else { - long stamp = stampedLock.writeLock(); - try { - testData = !testData; - counter++; - return testData; - } finally { - stampedLock.unlockWrite(stamp); - } - } - } - - @Benchmark - @Threads(16) - public boolean benchmarkOptimisticReadContention() { - // High contention scenario to test optimistic read fallback behavior - long stamp = stampedLock.tryOptimisticRead(); - boolean result = testData; - - // Simulate some computation - if (ThreadLocalRandom.current().nextInt(1000) == 0) { - Thread.yield(); // Occasionally yield to increase contention - } - - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); - try { - result = testData; - } finally { - stampedLock.unlockRead(stamp); - } - } - return result; - } -} \ No newline at end of file diff --git a/microbench/src/main/java/org/apache/pulsar/broker/delayed/bucket/MockBucketSnapshotStorage.java b/microbench/src/main/java/org/apache/pulsar/broker/delayed/bucket/MockBucketSnapshotStorage.java new file mode 100644 index 0000000000000..d220658fb2c6d --- /dev/null +++ b/microbench/src/main/java/org/apache/pulsar/broker/delayed/bucket/MockBucketSnapshotStorage.java @@ -0,0 +1,77 @@ +/* + * 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.bucket; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.pulsar.broker.delayed.proto.SnapshotMetadata; +import org.apache.pulsar.broker.delayed.proto.SnapshotSegment; + +public class MockBucketSnapshotStorage implements BucketSnapshotStorage { + + private final AtomicLong idGenerator = new AtomicLong(1); + private final Map snapshots = new ConcurrentHashMap<>(); + + @Override + public CompletableFuture createBucketSnapshot(SnapshotMetadata snapshotMetadata, + List bucketSnapshotSegments, + String bucketKey, String topicName, String cursorName) { + long id = idGenerator.getAndIncrement(); + snapshots.put(id, snapshotMetadata); + return CompletableFuture.completedFuture(id); + } + + @Override + public CompletableFuture getBucketSnapshotMetadata(long bucketId) { + SnapshotMetadata metadata = snapshots.get(bucketId); + return CompletableFuture.completedFuture(metadata); + } + + @Override + public CompletableFuture> getBucketSnapshotSegment(long bucketId, + long firstSegmentEntryId, + long lastSegmentEntryId) { + return CompletableFuture.completedFuture(Collections.emptyList()); + } + + @Override + public CompletableFuture getBucketSnapshotLength(long bucketId) { + return CompletableFuture.completedFuture(0L); + } + + @Override + public CompletableFuture deleteBucketSnapshot(long bucketId) { + snapshots.remove(bucketId); + return CompletableFuture.completedFuture(null); + } + + @Override + public void start() throws Exception { + // No-op + } + + @Override + public void close() throws Exception { + snapshots.clear(); + } +} diff --git a/microbench/src/main/java/org/apache/pulsar/broker/package-info.java b/microbench/src/main/java/org/apache/pulsar/broker/package-info.java new file mode 100644 index 0000000000000..a7620134f2f84 --- /dev/null +++ b/microbench/src/main/java/org/apache/pulsar/broker/package-info.java @@ -0,0 +1,27 @@ +/* + * 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. + */ + +/** + * Microbenchmarks for delayed message delivery bucket implementation. + * + *

This package contains JMH benchmarks for testing the performance + * characteristics of the BucketDelayedDeliveryTracker, particularly + * focusing on thread safety improvements with ReentrantReadWriteLock. + */ +package org.apache.pulsar.broker; \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java index bec5134c4f79a..b873f6a411127 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java @@ -29,7 +29,7 @@ @Slf4j public abstract class AbstractDelayedDeliveryTracker implements DelayedDeliveryTracker, TimerTask { - protected final AbstractPersistentDispatcherMultipleConsumers dispatcher; + protected final DelayedDeliveryContext context; // Reference to the shared (per-broker) timer for delayed delivery protected final Timer timer; @@ -49,23 +49,38 @@ public abstract class AbstractDelayedDeliveryTracker implements DelayedDeliveryT private final boolean isDelayedDeliveryDeliverAtTimeStrict; + private final Object timerStateLock = new Object(); + public AbstractDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict) { - this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict); + this(new DispatcherDelayedDeliveryContext(dispatcher), timer, tickTimeMillis, + Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict); } public AbstractDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, Clock clock, boolean isDelayedDeliveryDeliverAtTimeStrict) { - this.dispatcher = dispatcher; + this(new DispatcherDelayedDeliveryContext(dispatcher), timer, tickTimeMillis, + clock, isDelayedDeliveryDeliverAtTimeStrict); + } + + public AbstractDelayedDeliveryTracker(DelayedDeliveryContext context, Timer timer, + long tickTimeMillis, + boolean isDelayedDeliveryDeliverAtTimeStrict) { + this(context, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict); + } + + public AbstractDelayedDeliveryTracker(DelayedDeliveryContext context, Timer timer, + long tickTimeMillis, Clock clock, + boolean isDelayedDeliveryDeliverAtTimeStrict) { + this.context = context; this.timer = timer; this.tickTimeMillis = tickTimeMillis; this.clock = clock; this.isDelayedDeliveryDeliverAtTimeStrict = isDelayedDeliveryDeliverAtTimeStrict; } - /** * When {@link #isDelayedDeliveryDeliverAtTimeStrict} is false, we allow for early delivery by as much as the * {@link #tickTimeMillis} because it is a slight optimization to let messages skip going back into the delay @@ -89,72 +104,87 @@ public void resetTickTime(long tickTime) { protected void updateTimer() { if (getNumberOfDelayedMessages() == 0) { - if (timeout != null) { - currentTimeoutTarget = -1; - timeout.cancel(); - timeout = null; + synchronized (timerStateLock) { + if (timeout != null) { + currentTimeoutTarget = -1; + timeout.cancel(); + timeout = null; + } } return; } long timestamp = nextDeliveryTime(); - if (timestamp == currentTimeoutTarget) { - // The timer is already set to the correct target time - return; - } + synchronized (timerStateLock) { + if (timestamp == currentTimeoutTarget) { + // The timer is already set to the correct target time + return; + } - if (timeout != null) { - timeout.cancel(); - } + if (timeout != null) { + timeout.cancel(); + } - long now = clock.millis(); - long delayMillis = timestamp - now; + long now = clock.millis(); + long delayMillis = timestamp - now; + + if (delayMillis < 0) { + // There are messages that are already ready to be delivered. If + // the dispatcher is not getting them is because the consumer is + // either not connected or slow. + // We don't need to keep retriggering the timer. When the consumer + // catches up, the dispatcher will do the readMoreEntries() and + // get these messages + return; + } - if (delayMillis < 0) { - // There are messages that are already ready to be delivered. If - // the dispatcher is not getting them is because the consumer is - // either not connected or slow. - // We don't need to keep retriggering the timer. When the consumer - // catches up, the dispatcher will do the readMoreEntries() and - // get these messages - return; - } + // Compute the earliest time that we schedule the timer to run. + long remainingTickDelayMillis = lastTickRun + tickTimeMillis - now; + long calculatedDelayMillis = Math.max(delayMillis, remainingTickDelayMillis); - // Compute the earliest time that we schedule the timer to run. - long remainingTickDelayMillis = lastTickRun + tickTimeMillis - now; - long calculatedDelayMillis = Math.max(delayMillis, remainingTickDelayMillis); + if (log.isDebugEnabled()) { + log.debug("[{}] Start timer in {} millis", context.getName(), calculatedDelayMillis); + } - if (log.isDebugEnabled()) { - log.debug("[{}] Start timer in {} millis", dispatcher.getName(), calculatedDelayMillis); + // Even though we may delay longer than this timestamp because of the tick delay, we still track the + // current timeout with reference to the next message's timestamp. + currentTimeoutTarget = timestamp; + timeout = timer.newTimeout(this, calculatedDelayMillis, TimeUnit.MILLISECONDS); } + } - // Even though we may delay longer than this timestamp because of the tick delay, we still track the - // current timeout with reference to the next message's timestamp. - currentTimeoutTarget = timestamp; - timeout = timer.newTimeout(this, calculatedDelayMillis, TimeUnit.MILLISECONDS); + protected final void scheduleImmediateRun() { + synchronized (timerStateLock) { + if (timeout != null) { + timeout.cancel(); + } + timeout = timer.newTimeout(this, 0, TimeUnit.MILLISECONDS); + } } @Override public void run(Timeout timeout) throws Exception { if (log.isDebugEnabled()) { - log.debug("[{}] Timer triggered", dispatcher.getName()); + log.debug("[{}] Timer triggered", context.getName()); } if (timeout == null || timeout.isCancelled()) { return; } - synchronized (dispatcher) { + synchronized (timerStateLock) { lastTickRun = clock.millis(); currentTimeoutTarget = -1; this.timeout = null; - dispatcher.readMoreEntriesAsync(); } + context.triggerReadMoreEntries(); } @Override public void close() { - if (timeout != null) { - timeout.cancel(); - timeout = null; + synchronized (timerStateLock) { + if (timeout != null) { + timeout.cancel(); + timeout = null; + } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java index 93eb3ebbc77d5..078a1f8ad38ea 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java @@ -99,7 +99,18 @@ public DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleCon @VisibleForTesting BucketDelayedDeliveryTracker newTracker0(AbstractPersistentDispatcherMultipleConsumers dispatcher) throws RecoverDelayedDeliveryTrackerException { - return new BucketDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, + DelayedDeliveryContext context = new DispatcherDelayedDeliveryContext(dispatcher); + return new BucketDelayedDeliveryTracker(context, timer, tickTimeMillis, + isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket, + TimeUnit.SECONDS.toMillis(delayedDeliveryMaxTimeStepPerBucketSnapshotSegmentSeconds), + delayedDeliveryMaxIndexesPerBucketSnapshotSegment, delayedDeliveryMaxNumBuckets); + } + + @VisibleForTesting + BucketDelayedDeliveryTracker newTracker0(String dispatcherName, ManagedCursor cursor) + throws RecoverDelayedDeliveryTrackerException { + DelayedDeliveryContext context = new NoopDelayedDeliveryContext(dispatcherName, cursor); + return new BucketDelayedDeliveryTracker(context, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket, TimeUnit.SECONDS.toMillis(delayedDeliveryMaxTimeStepPerBucketSnapshotSegmentSeconds), delayedDeliveryMaxIndexesPerBucketSnapshotSegment, delayedDeliveryMaxNumBuckets); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryContext.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryContext.java new file mode 100644 index 0000000000000..fe200e18c0337 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryContext.java @@ -0,0 +1,30 @@ +/* + * 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 org.apache.bookkeeper.mledger.ManagedCursor; + +public interface DelayedDeliveryContext { + + String getName(); + + ManagedCursor getCursor(); + + void triggerReadMoreEntries(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DispatcherDelayedDeliveryContext.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DispatcherDelayedDeliveryContext.java new file mode 100644 index 0000000000000..91a8f4b234e77 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DispatcherDelayedDeliveryContext.java @@ -0,0 +1,48 @@ +/* + * 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 org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; + +public class DispatcherDelayedDeliveryContext implements DelayedDeliveryContext { + + private final AbstractPersistentDispatcherMultipleConsumers dispatcher; + + public DispatcherDelayedDeliveryContext(AbstractPersistentDispatcherMultipleConsumers dispatcher) { + this.dispatcher = dispatcher; + } + + @Override + public String getName() { + return dispatcher.getName(); + } + + @Override + public ManagedCursor getCursor() { + return dispatcher.getCursor(); + } + + @Override + public void triggerReadMoreEntries() { + synchronized (dispatcher) { + dispatcher.readMoreEntriesAsync(); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index ad5ab25fbbf6b..93b1079390f21 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicLong; import lombok.Getter; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; @@ -72,15 +73,39 @@ public class InMemoryDelayedDeliveryTracker extends AbstractDelayedDeliveryTrack long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, long fixedDelayDetectionLookahead) { - this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict, - fixedDelayDetectionLookahead); + this(new DispatcherDelayedDeliveryContext(dispatcher), timer, tickTimeMillis, Clock.systemUTC(), + isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); } public InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, Clock clock, boolean isDelayedDeliveryDeliverAtTimeStrict, long fixedDelayDetectionLookahead) { - super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); + this(new DispatcherDelayedDeliveryContext(dispatcher), timer, tickTimeMillis, clock, + isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); + } + + public InMemoryDelayedDeliveryTracker(String dispatcherName, ManagedCursor cursor, Timer timer, + long tickTimeMillis, Clock clock, + boolean isDelayedDeliveryDeliverAtTimeStrict, + long fixedDelayDetectionLookahead) { + this(new NoopDelayedDeliveryContext(dispatcherName, cursor), timer, tickTimeMillis, clock, + isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); + } + + public InMemoryDelayedDeliveryTracker(DelayedDeliveryContext context, Timer timer, + long tickTimeMillis, + boolean isDelayedDeliveryDeliverAtTimeStrict, + long fixedDelayDetectionLookahead) { + this(context, timer, tickTimeMillis, Clock.systemUTC(), + isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); + } + + public InMemoryDelayedDeliveryTracker(DelayedDeliveryContext context, Timer timer, + long tickTimeMillis, Clock clock, + boolean isDelayedDeliveryDeliverAtTimeStrict, + long fixedDelayDetectionLookahead) { + super(context, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); this.fixedDelayDetectionLookahead = fixedDelayDetectionLookahead; this.timestampPrecisionBitCnt = calculateTimestampPrecisionBitCnt(tickTimeMillis); } @@ -121,7 +146,7 @@ public boolean addMessage(long ledgerId, long entryId, long deliverAt) { } if (log.isDebugEnabled()) { - log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId, + log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", context.getName(), ledgerId, entryId, deliverAt - clock.millis()); } @@ -213,7 +238,7 @@ public NavigableSet getScheduledMessages(int maxMessages) { } if (log.isDebugEnabled()) { - log.debug("[{}] Get scheduled messages - found {}", dispatcher.getName(), positions.size()); + log.debug("[{}] Get scheduled messages - found {}", context.getName(), positions.size()); } if (delayedMessageMap.isEmpty()) { @@ -222,7 +247,7 @@ public NavigableSet getScheduledMessages(int maxMessages) { messagesHaveFixedDelay = true; if (delayedMessagesCount.get() != 0) { log.warn("[{}] Delayed message tracker is empty, but delayedMessagesCount is {}", - dispatcher.getName(), delayedMessagesCount.get()); + context.getName(), delayedMessagesCount.get()); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java index f8b8f5a8ba459..b0edfc455a527 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java @@ -23,6 +23,7 @@ import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; @@ -67,7 +68,15 @@ public DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleCon @VisibleForTesting InMemoryDelayedDeliveryTracker newTracker0(AbstractPersistentDispatcherMultipleConsumers dispatcher) { - return new InMemoryDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, + DelayedDeliveryContext context = new DispatcherDelayedDeliveryContext(dispatcher); + return new InMemoryDelayedDeliveryTracker(context, timer, tickTimeMillis, + isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); + } + + @VisibleForTesting + InMemoryDelayedDeliveryTracker newTracker0(String dispatcherName, ManagedCursor cursor) { + DelayedDeliveryContext context = new NoopDelayedDeliveryContext(dispatcherName, cursor); + return new InMemoryDelayedDeliveryTracker(context, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/NoopDelayedDeliveryContext.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/NoopDelayedDeliveryContext.java new file mode 100644 index 0000000000000..827fbd6cec553 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/NoopDelayedDeliveryContext.java @@ -0,0 +1,54 @@ +/* + * 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 java.util.concurrent.atomic.AtomicInteger; +import org.apache.bookkeeper.mledger.ManagedCursor; + +public class NoopDelayedDeliveryContext implements DelayedDeliveryContext { + + private final String name; + private final ManagedCursor cursor; + private final AtomicInteger triggerCount = new AtomicInteger(); + + public NoopDelayedDeliveryContext(String name, ManagedCursor cursor) { + this.name = name; + this.cursor = cursor; + } + + @Override + public String getName() { + return name; + } + + @Override + public ManagedCursor getCursor() { + return cursor; + } + + @Override + public void triggerReadMoreEntries() { + // no-op; for tests/JMH + triggerCount.incrementAndGet(); + } + + public int getTriggerCount() { + return triggerCount.get(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/Bucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/Bucket.java index a1693b1553d97..c355405846312 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/Bucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/Bucket.java @@ -137,7 +137,23 @@ CompletableFuture asyncSaveBucketSnapshot( List bucketSnapshotSegments) { final String bucketKey = bucket.bucketKey(); final String cursorName = Codec.decode(cursor.getName()); - final String topicName = dispatcherName.substring(0, dispatcherName.lastIndexOf(" / " + cursorName)); + final String suffix = " / " + cursorName; + final int suffixIndex = dispatcherName.lastIndexOf(suffix); + final String topicName; + if (suffixIndex >= 0) { + topicName = dispatcherName.substring(0, suffixIndex); + } else { + // Fallback for dispatcher names that don't follow the " / " pattern. + // This can happen in tests or benchmarks that use a simplified dispatcher name. + // Using the full dispatcherName as topicName avoids StringIndexOutOfBoundsException + // while still providing a meaningful identifier to the snapshot storage. + topicName = dispatcherName; + if (log.isDebugEnabled()) { + log.debug("Dispatcher name '{}' does not contain expected suffix '{}', " + + "using full dispatcherName as topic name", + dispatcherName, suffix); + } + } return executeWithRetry( () -> bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata, bucketSnapshotSegments, bucketKey, topicName, cursorName) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index 3f0fcc516571f..df5fe87f907d0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -40,10 +40,14 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.StampedLock; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; import javax.annotation.concurrent.ThreadSafe; import lombok.Getter; @@ -56,9 +60,13 @@ import org.apache.commons.lang3.mutable.MutableLong; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker; +import org.apache.pulsar.broker.delayed.DelayedDeliveryContext; +import org.apache.pulsar.broker.delayed.DispatcherDelayedDeliveryContext; +import org.apache.pulsar.broker.delayed.NoopDelayedDeliveryContext; import org.apache.pulsar.broker.delayed.proto.DelayedIndex; import org.apache.pulsar.broker.delayed.proto.SnapshotSegment; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; +import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; @@ -94,12 +102,9 @@ public static record SnapshotKey(long ledgerId, long entryId) {} private final AtomicLong numberDelayedMessages = new AtomicLong(0); - // Thread safety locks - private final StampedLock stampedLock = new StampedLock(); - @Getter @VisibleForTesting - private final MutableBucket lastMutableBucket; + private volatile MutableBucket lastMutableBucket; @Getter @VisibleForTesting @@ -115,6 +120,31 @@ public static record SnapshotKey(long ledgerId, long entryId) {} private CompletableFuture pendingLoad = null; + private final ExecutorService bucketSnapshotExecutor; + private final AtomicBoolean bucketSnapshotInProgress = new AtomicBoolean(false); + /** + * Bucket that is currently being sealed into an immutable bucket. + * + *

Lifecycle: + *

+ * + *

All access to this field must be done under {@link #writeLock} to keep bucket routing and sealing + * consistent across threads.

+ */ + private volatile MutableBucket bucketBeingSealed = null; + + private final Lock readLock; + private final Lock writeLock; + public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, @@ -122,19 +152,55 @@ public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumer long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) throws RecoverDelayedDeliveryTrackerException { - this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict, + this(new DispatcherDelayedDeliveryContext(dispatcher), timer, tickTimeMillis, Clock.systemUTC(), + isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, minIndexCountPerBucket, + timeStepPerBucketSnapshotSegmentInMillis, maxIndexesPerBucketSnapshotSegment, maxNumBuckets); + } + + public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, + Timer timer, long tickTimeMillis, Clock clock, + boolean isDelayedDeliveryDeliverAtTimeStrict, + BucketSnapshotStorage bucketSnapshotStorage, + long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, + int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) + throws RecoverDelayedDeliveryTrackerException { + this(new DispatcherDelayedDeliveryContext(dispatcher), timer, tickTimeMillis, clock, + isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, minIndexCountPerBucket, + timeStepPerBucketSnapshotSegmentInMillis, maxIndexesPerBucketSnapshotSegment, maxNumBuckets); + } + + public BucketDelayedDeliveryTracker(String dispatcherName, ManagedCursor cursor, + Timer timer, long tickTimeMillis, + boolean isDelayedDeliveryDeliverAtTimeStrict, + BucketSnapshotStorage bucketSnapshotStorage, + long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, + int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) + throws RecoverDelayedDeliveryTrackerException { + this(new NoopDelayedDeliveryContext(dispatcherName, cursor), timer, tickTimeMillis, Clock.systemUTC(), + isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, minIndexCountPerBucket, + timeStepPerBucketSnapshotSegmentInMillis, maxIndexesPerBucketSnapshotSegment, maxNumBuckets); + } + + public BucketDelayedDeliveryTracker(DelayedDeliveryContext context, + Timer timer, long tickTimeMillis, + boolean isDelayedDeliveryDeliverAtTimeStrict, + BucketSnapshotStorage bucketSnapshotStorage, + long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, + int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) + throws RecoverDelayedDeliveryTrackerException { + this(context, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegmentInMillis, maxIndexesPerBucketSnapshotSegment, maxNumBuckets); } - public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, + public BucketDelayedDeliveryTracker(DelayedDeliveryContext context, Timer timer, long tickTimeMillis, Clock clock, boolean isDelayedDeliveryDeliverAtTimeStrict, BucketSnapshotStorage bucketSnapshotStorage, long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) throws RecoverDelayedDeliveryTrackerException { - super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); + super(context, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); this.minIndexCountPerBucket = minIndexCountPerBucket; this.timeStepPerBucketSnapshotSegmentInMillis = timeStepPerBucketSnapshotSegmentInMillis; this.maxIndexesPerBucketSnapshotSegment = maxIndexesPerBucketSnapshotSegment; @@ -143,11 +209,15 @@ public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumer this.immutableBuckets = TreeRangeMap.create(); this.snapshotSegmentLastIndexMap = new ConcurrentHashMap<>(); this.lastMutableBucket = - new MutableBucket(dispatcher.getName(), dispatcher.getCursor(), FutureUtil.Sequencer.create(), + new MutableBucket(context.getName(), context.getCursor(), FutureUtil.Sequencer.create(), bucketSnapshotStorage); this.stats = new BucketDelayedMessageIndexStats(); + ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); + this.readLock = rwLock.readLock(); + this.writeLock = rwLock.writeLock(); + bucketSnapshotExecutor = Executors.newSingleThreadScheduledExecutor( + new ExecutorProvider.ExtendedThreadFactory(context.getName() + "bucket-creation")); - // Close the tracker if failed to recover. try { long recoveredMessages = recoverBucketSnapshot(); this.numberDelayedMessages.set(recoveredMessages); @@ -162,7 +232,7 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT Map cursorProperties = cursor.getCursorProperties(); if (MapUtils.isEmpty(cursorProperties)) { log.info("[{}] Recover delayed message index bucket snapshot finish, don't find bucket snapshot", - dispatcher.getName()); + context.getName()); return 0; } FutureUtil.Sequencer sequencer = this.lastMutableBucket.getSequencer(); @@ -172,7 +242,7 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT String[] keys = key.split(DELIMITER); checkArgument(keys.length == 3); ImmutableBucket immutableBucket = - new ImmutableBucket(dispatcher.getName(), cursor, sequencer, + new ImmutableBucket(context.getName(), cursor, sequencer, this.lastMutableBucket.bucketSnapshotStorage, Long.parseLong(keys[1]), Long.parseLong(keys[2])); putAndCleanOverlapRange(Range.closed(immutableBucket.startLedgerId, immutableBucket.endLedgerId), @@ -183,7 +253,7 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT Map, ImmutableBucket> immutableBucketMap = immutableBuckets.asMapOfRanges(); if (immutableBucketMap.isEmpty()) { log.info("[{}] Recover delayed message index bucket snapshot finish, don't find bucket snapshot", - dispatcher.getName()); + context.getName()); return 0; } @@ -197,7 +267,7 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT try { FutureUtil.waitForAll(futures.values()).get(AsyncOperationTimeoutSeconds * 5, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { - log.error("[{}] Failed to recover delayed message index bucket snapshot.", dispatcher.getName(), e); + log.error("[{}] Failed to recover delayed message index bucket snapshot.", context.getName(), e); if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } @@ -238,7 +308,7 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT }); log.info("[{}] Recover delayed message index bucket snapshot finish, buckets: {}, numberDelayedMessages: {}", - dispatcher.getName(), immutableBucketMap.size(), numberDelayedMessages.getValue()); + context.getName(), immutableBucketMap.size(), numberDelayedMessages.getValue()); return numberDelayedMessages.getValue(); } @@ -292,11 +362,14 @@ private synchronized void putAndCleanOverlapRange(Range range, ImmutableBu @Override public void run(Timeout timeout) throws Exception { - synchronized (this) { + writeLock.lock(); + try { if (timeout == null || timeout.isCancelled()) { return; } lastMutableBucket.moveScheduledMessageToSharedQueue(getCutoffTime(), sharedBucketPriorityQueue); + } finally { + writeLock.unlock(); } super.run(timeout); } @@ -326,7 +399,7 @@ private void afterCreateImmutableBucket(Pair immu if (ex == null) { immutableBucket.setSnapshotSegments(null); immutableBucket.asyncUpdateSnapshotLength(); - log.info("[{}] Create bucket snapshot finish, bucketKey: {}", dispatcher.getName(), + log.info("[{}] Create bucket snapshot finish, bucketKey: {}", context.getName(), immutableBucket.bucketKey()); stats.recordSuccessEvent(BucketDelayedMessageIndexStats.Type.create, @@ -335,12 +408,13 @@ private void afterCreateImmutableBucket(Pair immu return bucketId; } - log.error("[{}] Failed to create bucket snapshot, bucketKey: {}", dispatcher.getName(), + log.error("[{}] Failed to create bucket snapshot, bucketKey: {}", context.getName(), immutableBucket.bucketKey(), ex); stats.recordFailEvent(BucketDelayedMessageIndexStats.Type.create); // Put indexes back into the shared queue and downgrade to memory mode - synchronized (BucketDelayedDeliveryTracker.this) { + writeLock.lock(); + try { immutableBucket.getSnapshotSegments().ifPresent(snapshotSegments -> { for (SnapshotSegment snapshotSegment : snapshotSegments) { for (DelayedIndex delayedIndex : snapshotSegment.getIndexesList()) { @@ -356,6 +430,8 @@ private void afterCreateImmutableBucket(Pair immu Range.closed(immutableBucket.startLedgerId, immutableBucket.endLedgerId)); snapshotSegmentLastIndexMap.remove( new SnapshotKey(lastDelayedIndex.getLedgerId(), lastDelayedIndex.getEntryId())); + } finally { + writeLock.unlock(); } return INVALID_BUCKET_ID; }); @@ -364,60 +440,109 @@ private void afterCreateImmutableBucket(Pair immu } } + /** + * Add a delayed message to the tracker. + * + *

Routing rules (under {@link #writeLock} for the mutating part): + *

    + *
  • If the message already exists, it is ignored.
  • + *
  • If the delivery time is before the cutoff, it is dropped.
  • + *
  • If the current {@link #lastMutableBucket} has accumulated enough indexes and the message + * ledger id is strictly after its range, a snapshot is triggered: + *
      + *
    • The current {@code lastMutableBucket} is moved to {@link #bucketBeingSealed}.
    • + *
    • A fresh {@code lastMutableBucket} is created for subsequent messages.
    • + *
    • Sealing and persistence happen asynchronously in + * {@link #createBucketSnapshotAsync(MutableBucket)}.
    • + *
    + *
  • + *
  • Messages whose ledger id falls into the range of a bucket that is currently being sealed, or + * into an already immutable bucket, or before the current {@code lastMutableBucket} range, + * are routed directly to {@link #sharedBucketPriorityQueue} and tracked only via the bitmap + * of the current {@code lastMutableBucket}. This mirrors the original behaviour where + * messages for closed ranges are not re-added to mutable buckets.
  • + *
  • All remaining messages are added to {@code lastMutableBucket}.
  • + *
+ */ @Override - public synchronized boolean addMessage(long ledgerId, long entryId, long deliverAt) { - if (containsMessage(ledgerId, entryId)) { - return true; - } + public boolean addMessage(long ledgerId, long entryId, long deliverAt) { + readLock.lock(); + try { + if (containsMessageUnsafe(ledgerId, entryId)) { + return true; + } - if (deliverAt < 0 || deliverAt <= getCutoffTime()) { - return false; + if (deliverAt < 0 || deliverAt <= getCutoffTime()) { + return false; + } + } finally { + readLock.unlock(); } - boolean existBucket = findImmutableBucket(ledgerId).isPresent(); - - // Create bucket snapshot - if (!existBucket && ledgerId > lastMutableBucket.endLedgerId - && lastMutableBucket.size() >= minIndexCountPerBucket - && !lastMutableBucket.isEmpty()) { - long createStartTime = System.currentTimeMillis(); - stats.recordTriggerEvent(BucketDelayedMessageIndexStats.Type.create); - Pair immutableBucketDelayedIndexPair = - lastMutableBucket.sealBucketAndAsyncPersistent( - this.timeStepPerBucketSnapshotSegmentInMillis, - this.maxIndexesPerBucketSnapshotSegment, - this.sharedBucketPriorityQueue); - afterCreateImmutableBucket(immutableBucketDelayedIndexPair, createStartTime); - lastMutableBucket.resetLastMutableBucketRange(); + writeLock.lock(); + try { + // Double check + if (containsMessageUnsafe(ledgerId, entryId)) { + return true; + } - if (maxNumBuckets > 0 && immutableBuckets.asMapOfRanges().size() > maxNumBuckets) { - asyncMergeBucketSnapshot(); + if (deliverAt <= getCutoffTime()) { + return false; } - } - 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); - lastMutableBucket.putIndexBit(ledgerId, entryId); - } else { - checkArgument(ledgerId >= lastMutableBucket.endLedgerId); - lastMutableBucket.addMessage(ledgerId, entryId, deliverAt); - } + final MutableBucket sealingBucket = this.bucketBeingSealed; + final boolean sealingBucketContainsLedger = sealingBucket != null + && ledgerId >= sealingBucket.startLedgerId + && ledgerId <= sealingBucket.endLedgerId; + + boolean existBucket = findImmutableBucket(ledgerId).isPresent(); + + if (!sealingBucketContainsLedger + && !existBucket + && ledgerId > lastMutableBucket.endLedgerId + && lastMutableBucket.size() >= minIndexCountPerBucket + && !lastMutableBucket.isEmpty() + && bucketSnapshotInProgress.compareAndSet(false, true)) { + // Create bucket snapshot using current lastMutableBucket as the bucket to seal. + final MutableBucket bucketToSeal = this.lastMutableBucket; + this.bucketBeingSealed = bucketToSeal; + this.lastMutableBucket = new MutableBucket(context.getName(), context.getCursor(), + FutureUtil.Sequencer.create(), bucketToSeal.getBucketSnapshotStorage()); + bucketSnapshotExecutor.execute(() -> { + try { + createBucketSnapshotAsync(bucketToSeal); + } finally { + bucketSnapshotInProgress.set(false); + } + }); + } - numberDelayedMessages.incrementAndGet(); + if (sealingBucketContainsLedger || ledgerId < lastMutableBucket.startLedgerId || existBucket) { + // If message belongs to a bucket currently being sealed, or an existing immutable bucket, + // or has ledgerId smaller than current lastMutableBucket range, we put it directly into + // the shared queue and track its index bit in the current mutable bucket. + sharedBucketPriorityQueue.add(deliverAt, ledgerId, entryId); + lastMutableBucket.putIndexBit(ledgerId, entryId); + } else { + checkArgument(ledgerId >= lastMutableBucket.endLedgerId); + lastMutableBucket.addMessage(ledgerId, entryId, deliverAt); + } - if (log.isDebugEnabled()) { - log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId, - deliverAt - clock.millis()); - } + numberDelayedMessages.incrementAndGet(); - updateTimer(); + if (log.isDebugEnabled()) { + log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", context.getName(), ledgerId, entryId, + deliverAt - clock.millis()); + } - return true; + updateTimer(); + return true; + } finally { + writeLock.unlock(); + } } - private synchronized List selectMergedBuckets(final List values, int mergeNum) { + private List selectMergedBuckets(final List values, int mergeNum) { checkArgument(mergeNum < values.size()); long minNumberMessages = Long.MAX_VALUE; long minScheduleTimestamp = Long.MAX_VALUE; @@ -454,49 +579,57 @@ private synchronized List selectMergedBuckets(final List asyncMergeBucketSnapshot() { - List immutableBucketList = immutableBuckets.asMapOfRanges().values().stream().toList(); - List toBeMergeImmutableBuckets = selectMergedBuckets(immutableBucketList, MAX_MERGE_NUM); + private CompletableFuture asyncMergeBucketSnapshot() { + writeLock.lock(); + try { + List immutableBucketList = immutableBuckets.asMapOfRanges().values().stream().toList(); + List toBeMergeImmutableBuckets = selectMergedBuckets(immutableBucketList, MAX_MERGE_NUM); - if (toBeMergeImmutableBuckets.isEmpty()) { - log.warn("[{}] Can't find able merged buckets", dispatcher.getName()); - return CompletableFuture.completedFuture(null); - } + if (toBeMergeImmutableBuckets.isEmpty()) { + log.warn("[{}] Can't find able merged buckets", context.getName()); + return CompletableFuture.completedFuture(null); + } - final String bucketsStr = toBeMergeImmutableBuckets.stream().map(Bucket::bucketKey).collect( - Collectors.joining(",")).replaceAll(DELAYED_BUCKET_KEY_PREFIX + "_", ""); - if (log.isDebugEnabled()) { - log.info("[{}] Merging bucket snapshot, bucketKeys: {}", dispatcher.getName(), bucketsStr); - } + final String bucketsStr = toBeMergeImmutableBuckets.stream().map(Bucket::bucketKey).collect( + Collectors.joining(",")).replaceAll(DELAYED_BUCKET_KEY_PREFIX + "_", ""); + if (log.isDebugEnabled()) { + log.info("[{}] Merging bucket snapshot, bucketKeys: {}", context.getName(), bucketsStr); + } - for (ImmutableBucket immutableBucket : toBeMergeImmutableBuckets) { - immutableBucket.merging = true; - } + for (ImmutableBucket immutableBucket : toBeMergeImmutableBuckets) { + immutableBucket.merging = true; + } - long mergeStartTime = System.currentTimeMillis(); - stats.recordTriggerEvent(BucketDelayedMessageIndexStats.Type.merge); - return asyncMergeBucketSnapshot(toBeMergeImmutableBuckets).whenComplete((__, ex) -> { - synchronized (this) { - for (ImmutableBucket immutableBucket : toBeMergeImmutableBuckets) { - immutableBucket.merging = false; + long mergeStartTime = System.currentTimeMillis(); + stats.recordTriggerEvent(BucketDelayedMessageIndexStats.Type.merge); + return asyncMergeBucketSnapshot(toBeMergeImmutableBuckets).whenComplete((__, ex) -> { + writeLock.lock(); + try { + for (ImmutableBucket immutableBucket : toBeMergeImmutableBuckets) { + immutableBucket.merging = false; + } + } finally { + writeLock.unlock(); } - } - if (ex != null) { - log.error("[{}] Failed to merge bucket snapshot, bucketKeys: {}", - dispatcher.getName(), bucketsStr, ex); + if (ex != null) { + log.error("[{}] Failed to merge bucket snapshot, bucketKeys: {}", + context.getName(), bucketsStr, ex); - stats.recordFailEvent(BucketDelayedMessageIndexStats.Type.merge); - } else { - log.info("[{}] Merge bucket snapshot finish, bucketKeys: {}, bucketNum: {}", - dispatcher.getName(), bucketsStr, immutableBuckets.asMapOfRanges().size()); + stats.recordFailEvent(BucketDelayedMessageIndexStats.Type.merge); + } else { + log.info("[{}] Merge bucket snapshot finish, bucketKeys: {}, bucketNum: {}", + context.getName(), bucketsStr, immutableBuckets.asMapOfRanges().size()); - stats.recordSuccessEvent(BucketDelayedMessageIndexStats.Type.merge, - System.currentTimeMillis() - mergeStartTime); - } - }); + stats.recordSuccessEvent(BucketDelayedMessageIndexStats.Type.merge, + System.currentTimeMillis() - mergeStartTime); + } + }); + } finally { + writeLock.unlock(); + } } - private synchronized CompletableFuture asyncMergeBucketSnapshot(List buckets) { + private CompletableFuture asyncMergeBucketSnapshot(List buckets) { List> createFutures = buckets.stream().map(bucket -> bucket.getSnapshotCreateFuture().orElse(NULL_LONG_PROMISE)) .toList(); @@ -515,7 +648,8 @@ private synchronized CompletableFuture asyncMergeBucketSnapshot(List { - synchronized (BucketDelayedDeliveryTracker.this) { + writeLock.lock(); + try { long createStartTime = System.currentTimeMillis(); stats.recordTriggerEvent(BucketDelayedMessageIndexStats.Type.create); Pair immutableBucketDelayedIndexPair = @@ -526,6 +660,12 @@ private synchronized CompletableFuture asyncMergeBucketSnapshot(List delayedIndexBitMap = new HashMap<>(buckets.get(0).getDelayedIndexBitMap()); @@ -560,38 +700,119 @@ private synchronized CompletableFuture asyncMergeBucketSnapshot(List 0 && nextDeliveryTime() <= cutoffTime; - if (!hasMessageAvailable) { - updateTimer(); + /** + * Seal the given mutable bucket into an immutable one and persist its snapshot. + * + *

This method is always executed on {@link #bucketSnapshotExecutor} and never mutates the + * actively written {@link #lastMutableBucket}. The {@code bucketToSeal} instance is detached + * from writes before this method is scheduled.

+ * + *

On success, {@link #afterCreateImmutableBucket(Pair, long)} is called under the + * {@link #writeLock}, the new immutable bucket is registered in {@link #immutableBuckets}, + * and {@link #bucketBeingSealed} is cleared. On failure or empty bucket, only the + * {@code bucketBeingSealed} state is cleared.

+ * + * @param bucketToSeal the mutable bucket that was selected under the write lock when the + * snapshot was triggered; it must not be modified by callers afterwards + */ + private void createBucketSnapshotAsync(MutableBucket bucketToSeal) { + if (bucketToSeal == null) { + return; } - return hasMessageAvailable; - } - @Override - protected long nextDeliveryTime() { - // Use optimistic read for frequently called method - long stamp = stampedLock.tryOptimisticRead(); - long result = nextDeliveryTimeUnsafe(); + try { + if (bucketToSeal.isEmpty()) { + clearBucketBeingSealed(bucketToSeal); + return; + } + long createStartTime = System.currentTimeMillis(); + stats.recordTriggerEvent(BucketDelayedMessageIndexStats.Type.create); + Pair immutableBucketDelayedIndexPair = + bucketToSeal.sealBucketAndAsyncPersistent( + this.timeStepPerBucketSnapshotSegmentInMillis, + this.maxIndexesPerBucketSnapshotSegment, + this.sharedBucketPriorityQueue); + if (immutableBucketDelayedIndexPair == null) { + clearBucketBeingSealed(bucketToSeal); + return; + } - if (!stampedLock.validate(stamp)) { - stamp = stampedLock.readLock(); + writeLock.lock(); try { - result = nextDeliveryTimeUnsafe(); + afterCreateImmutableBucket(immutableBucketDelayedIndexPair, createStartTime); + clearBucketBeingSealedUnsafe(bucketToSeal); + + if (maxNumBuckets > 0 && immutableBuckets.asMapOfRanges().size() > maxNumBuckets) { + asyncMergeBucketSnapshot(); + } } finally { - stampedLock.unlockRead(stamp); + writeLock.unlock(); } + } catch (Throwable t) { + log.error("[{}] Failed to create bucket snapshot", context.getName(), t); + clearBucketBeingSealed(bucketToSeal); + } + } + + /** + * Clear {@link #bucketBeingSealed} if it still refers to the given bucket. + * + *

This method acquires the {@link #writeLock}. Use + * {@link #clearBucketBeingSealedUnsafe(MutableBucket)} instead when the caller already + * holds the write lock.

+ */ + private void clearBucketBeingSealed(MutableBucket bucketToSeal) { + writeLock.lock(); + try { + clearBucketBeingSealedUnsafe(bucketToSeal); + } finally { + writeLock.unlock(); + } + } + + /** + * Clear {@link #bucketBeingSealed} without acquiring the {@link #writeLock}. + * + *

Callers must already hold {@link #writeLock} when invoking this method.

+ */ + private void clearBucketBeingSealedUnsafe(MutableBucket bucketToSeal) { + if (this.bucketBeingSealed == bucketToSeal) { + this.bucketBeingSealed = null; + } + } + + @Override + public boolean hasMessageAvailable() { + readLock.lock(); + try { + long cutoffTime = getCutoffTime(); + + boolean hasMessageAvailable = getNumberOfDelayedMessages() > 0 && nextDeliveryTimeUnsafe() <= cutoffTime; + if (!hasMessageAvailable) { + updateTimer(); + } + return hasMessageAvailable; + } finally { + readLock.unlock(); + } + } + + @Override + protected long nextDeliveryTime() { + readLock.lock(); + try { + return nextDeliveryTimeUnsafe(); + } finally { + readLock.unlock(); } - return result; } private long nextDeliveryTimeUnsafe() { @@ -616,126 +837,119 @@ public long getBufferMemoryUsage() { } @Override - public synchronized NavigableSet getScheduledMessages(int maxMessages) { - if (!checkPendingLoadDone()) { + public NavigableSet getScheduledMessages(int maxMessages) { + if (pendingLoad != null && !pendingLoad.isDone()) { if (log.isDebugEnabled()) { log.debug("[{}] Skip getScheduledMessages to wait for bucket snapshot load finish.", - dispatcher.getName()); + context.getName()); } return Collections.emptyNavigableSet(); } long cutoffTime = getCutoffTime(); - lastMutableBucket.moveScheduledMessageToSharedQueue(cutoffTime, sharedBucketPriorityQueue); - NavigableSet 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(); - - SnapshotKey snapshotKey = new SnapshotKey(ledgerId, entryId); - - ImmutableBucket bucket = snapshotSegmentLastIndexMap.get(snapshotKey); - if (bucket != null && immutableBuckets.asMapOfRanges().containsValue(bucket)) { - // All message of current snapshot segment are scheduled, try load next snapshot segment - if (bucket.merging) { - log.info("[{}] Skip load to wait for bucket snapshot merge finish, bucketKey:{}", - dispatcher.getName(), bucket.bucketKey()); + writeLock.lock(); + try { + lastMutableBucket.moveScheduledMessageToSharedQueue(cutoffTime, sharedBucketPriorityQueue); + while (positions.size() < maxMessages && !sharedBucketPriorityQueue.isEmpty()) { + if (sharedBucketPriorityQueue.peekN1() > cutoffTime) { + // All remaining messages are scheduled for the future break; } + long ledgerId = sharedBucketPriorityQueue.peekN2(); + long entryId = sharedBucketPriorityQueue.peekN3(); + // Check if this message is a trigger to load the next snapshot segment + SnapshotKey snapshotKey = new SnapshotKey(ledgerId, entryId); + ImmutableBucket bucket = snapshotSegmentLastIndexMap.get(snapshotKey); + if (bucket != null && immutableBuckets.asMapOfRanges().containsValue(bucket)) { + // All message of current snapshot segment are scheduled, try load next snapshot segment + if (bucket.merging) { + log.info("[{}] Skip load to wait for bucket snapshot merge finish, bucketKey:{}", + context.getName(), bucket.bucketKey()); + break; + } - final int preSegmentEntryId = bucket.currentSegmentEntryId; - if (log.isDebugEnabled()) { - log.debug("[{}] Loading next bucket snapshot segment, bucketKey: {}, nextSegmentEntryId: {}", - dispatcher.getName(), bucket.bucketKey(), preSegmentEntryId + 1); - } - boolean createFutureDone = bucket.getSnapshotCreateFuture().orElse(NULL_LONG_PROMISE).isDone(); - if (!createFutureDone) { - log.info("[{}] Skip load to wait for bucket snapshot create finish, bucketKey:{}", - dispatcher.getName(), bucket.bucketKey()); + // This is the last message of a segment. We need to load the next one. + // Trigger the load and stop processing more messages in this run. + // The positions collected so far will be returned. + triggerAsyncLoadBucketSnapshot(bucket, snapshotKey); break; } + // If it's a regular message (or in memory-only mode), process it. + sharedBucketPriorityQueue.pop(); // Consume the message from the queue + positions.add(PositionFactory.create(ledgerId, entryId)); + removeIndexBit(ledgerId, entryId); + } + if (!positions.isEmpty()) { + numberDelayedMessages.addAndGet(-positions.size()); + updateTimer(); + } + } finally { + writeLock.unlock(); + } + + return positions; + } + + private void triggerAsyncLoadBucketSnapshot(ImmutableBucket bucketToLoad, SnapshotKey snapshotKeyToLoad) { + final int preSegmentEntryId = bucketToLoad.currentSegmentEntryId; + if (log.isDebugEnabled()) { + log.debug("[{}] Loading next bucket snapshot segment, bucketKey: {}, nextSegmentEntryId: {}", + context.getName(), bucketToLoad.bucketKey(), preSegmentEntryId + 1); + } + + boolean createFutureDone = bucketToLoad.getSnapshotCreateFuture().orElse(NULL_LONG_PROMISE).isDone(); + if (!createFutureDone) { + log.info("[{}] Skip load to wait for bucket snapshot create finish, bucketKey:{}", + context.getName(), bucketToLoad.bucketKey()); + return; + } - long loadStartTime = System.currentTimeMillis(); - stats.recordTriggerEvent(BucketDelayedMessageIndexStats.Type.load); - CompletableFuture loadFuture = pendingLoad = bucket.asyncLoadNextBucketSnapshotEntry() - .thenAccept(indexList -> { - synchronized (BucketDelayedDeliveryTracker.this) { - this.snapshotSegmentLastIndexMap.remove(snapshotKey); + long loadStartTime = System.currentTimeMillis(); + stats.recordTriggerEvent(BucketDelayedMessageIndexStats.Type.load); + CompletableFuture loadFuture = pendingLoad = bucketToLoad.asyncLoadNextBucketSnapshotEntry() + .thenAccept(indexList -> { + writeLock.lock(); + try { + this.snapshotSegmentLastIndexMap.remove(snapshotKeyToLoad); if (CollectionUtils.isEmpty(indexList)) { immutableBuckets.asMapOfRanges() - .remove(Range.closed(bucket.startLedgerId, bucket.endLedgerId)); - bucket.asyncDeleteBucketSnapshot(stats); + .remove(Range.closed(bucketToLoad.startLedgerId, bucketToLoad.endLedgerId)); + bucketToLoad.asyncDeleteBucketSnapshot(stats); return; } DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1); this.snapshotSegmentLastIndexMap.put( new SnapshotKey(lastDelayedIndex.getLedgerId(), lastDelayedIndex.getEntryId()), - bucket); + bucketToLoad); for (DelayedIndex index : indexList) { sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(), index.getEntryId()); } + } finally { + writeLock.unlock(); } }).whenComplete((__, ex) -> { if (ex != null) { // Back bucket state - bucket.setCurrentSegmentEntryId(preSegmentEntryId); + bucketToLoad.setCurrentSegmentEntryId(preSegmentEntryId); log.error("[{}] Failed to load bucket snapshot segment, bucketKey: {}, segmentEntryId: {}", - dispatcher.getName(), bucket.bucketKey(), preSegmentEntryId + 1, ex); + context.getName(), bucketToLoad.bucketKey(), preSegmentEntryId + 1, ex); stats.recordFailEvent(BucketDelayedMessageIndexStats.Type.load); } else { log.info("[{}] Load next bucket snapshot segment finish, bucketKey: {}, segmentEntryId: {}", - dispatcher.getName(), bucket.bucketKey(), - (preSegmentEntryId == bucket.lastSegmentEntryId) ? "-1" : preSegmentEntryId + 1); + context.getName(), bucketToLoad.bucketKey(), + (preSegmentEntryId == bucketToLoad.lastSegmentEntryId) ? "-1" : preSegmentEntryId + 1); stats.recordSuccessEvent(BucketDelayedMessageIndexStats.Type.load, System.currentTimeMillis() - loadStartTime); } - synchronized (this) { - if (timeout != null) { - timeout.cancel(); - } - timeout = timer.newTimeout(this, 0, TimeUnit.MILLISECONDS); - } + scheduleImmediateRun(); }); - - if (!checkPendingLoadDone() || loadFuture.isCompletedExceptionally()) { - break; - } - } - - positions.add(PositionFactory.create(ledgerId, entryId)); - - sharedBucketPriorityQueue.pop(); - removeIndexBit(ledgerId, entryId); - - --n; - numberDelayedMessages.decrementAndGet(); - } - - updateTimer(); - - return positions; - } - - private synchronized boolean checkPendingLoadDone() { - if (pendingLoad == null || pendingLoad.isDone()) { - pendingLoad = null; - return true; - } - return false; } @Override @@ -744,26 +958,61 @@ public boolean shouldPauseAllDeliveries() { } @Override - public synchronized CompletableFuture clear() { - CompletableFuture future = cleanImmutableBuckets(); - sharedBucketPriorityQueue.clear(); - lastMutableBucket.clear(); - snapshotSegmentLastIndexMap.clear(); - numberDelayedMessages.set(0); - return future; + public CompletableFuture clear() { + writeLock.lock(); + try { + CompletableFuture future = cleanImmutableBuckets(); + sharedBucketPriorityQueue.clear(); + lastMutableBucket.clear(); + snapshotSegmentLastIndexMap.clear(); + numberDelayedMessages.set(0); + return future; + } finally { + writeLock.unlock(); + } } @Override - public synchronized void close() { - super.close(); - lastMutableBucket.close(); - sharedBucketPriorityQueue.close(); + public void close() { + writeLock.lock(); + try { + super.close(); + lastMutableBucket.close(); + sharedBucketPriorityQueue.close(); + } finally { + writeLock.unlock(); + } + bucketSnapshotExecutor.shutdown(); + try { + if (!bucketSnapshotExecutor.awaitTermination(AsyncOperationTimeoutSeconds, TimeUnit.SECONDS)) { + log.warn("[{}] bucketSnapshotExecutor did not terminate in the specified time.", + context.getName()); + } + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + log.warn("[{}] Interrupted while waiting for bucketSnapshotExecutor to terminate.", + context.getName(), ie); + } + + List> completableFutures = Collections.emptyList(); + writeLock.lock(); + try { + try { + completableFutures = immutableBuckets.asMapOfRanges().values().stream() + .map(bucket -> bucket.getSnapshotCreateFuture().orElse(NULL_LONG_PROMISE)).toList(); + } catch (Exception e) { + log.warn("[{}] Failed wait to snapshot generate", context.getName(), e); + } + } finally { + writeLock.unlock(); + } + try { - List> completableFutures = immutableBuckets.asMapOfRanges().values().stream() - .map(bucket -> bucket.getSnapshotCreateFuture().orElse(NULL_LONG_PROMISE)).toList(); - FutureUtil.waitForAll(completableFutures).get(AsyncOperationTimeoutSeconds, TimeUnit.SECONDS); + if (!completableFutures.isEmpty()) { + FutureUtil.waitForAll(completableFutures).get(AsyncOperationTimeoutSeconds, TimeUnit.SECONDS); + } } catch (Exception e) { - log.warn("[{}] Failed wait to snapshot generate", dispatcher.getName(), e); + log.warn("[{}] Failed wait to snapshot generate", context.getName(), e); } } @@ -789,21 +1038,12 @@ private boolean removeIndexBit(long ledgerId, long entryId) { } public boolean containsMessage(long ledgerId, long entryId) { - // Try optimistic read first for best performance - long stamp = stampedLock.tryOptimisticRead(); - boolean result = containsMessageUnsafe(ledgerId, entryId); - - - if (!stampedLock.validate(stamp)) { - // Fall back to read lock if validation fails - stamp = stampedLock.readLock(); - try { - result = containsMessageUnsafe(ledgerId, entryId); - } finally { - stampedLock.unlockRead(stamp); - } + readLock.lock(); + try { + return containsMessageUnsafe(ledgerId, entryId); + } finally { + readLock.unlock(); } - return result; } private boolean containsMessageUnsafe(long ledgerId, long entryId) { @@ -815,7 +1055,6 @@ private boolean containsMessageUnsafe(long ledgerId, long entryId) { .orElse(false); } - public Map genTopicMetricMap() { stats.recordNumOfBuckets(immutableBuckets.asMapOfRanges().size() + 1); stats.recordDelayedMessageIndexLoaded(this.sharedBucketPriorityQueue.size() + this.lastMutableBucket.size()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/CombinedSegmentDelayedIndexQueue.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/CombinedSegmentDelayedIndexQueue.java index 006938e9ed271..526b7d7532729 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/CombinedSegmentDelayedIndexQueue.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/CombinedSegmentDelayedIndexQueue.java @@ -48,7 +48,23 @@ static class Node { private CombinedSegmentDelayedIndexQueue(List> segmentLists) { this.kpq = new PriorityQueue<>(segmentLists.size(), COMPARATOR_NODE); for (List segmentList : segmentLists) { - Node node = new Node(segmentList, 0, 0); + if (segmentList == null || segmentList.isEmpty()) { + // Skip empty segment lists, there is nothing to merge from them. + continue; + } + + // Advance to the first non-empty segment in this list. + int segmentListCursor = 0; + while (segmentListCursor < segmentList.size() + && segmentList.get(segmentListCursor).getIndexesCount() == 0) { + segmentListCursor++; + } + if (segmentListCursor >= segmentList.size()) { + // All segments are empty, skip this list entirely. + continue; + } + + Node node = new Node(segmentList, segmentListCursor, 0); kpq.offer(node); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java index 6ff98fa7f7004..fe2d60bbf49b5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java @@ -40,6 +40,7 @@ import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -184,28 +185,34 @@ public void testContainsMessage(BucketDelayedDeliveryTracker tracker) { @Test(dataProvider = "delayedTracker", invocationCount = 10) public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) throws Exception { - for (int i = 1; i <= 100; i++) { + final int minIndexCountPerBucket = 5; + final int messagesToSnapshot = 100; + final int triggerMessageCount = messagesToSnapshot + 1; + for (int i = 1; i <= triggerMessageCount; i++) { tracker.addMessage(i, i, i * 10); + boolean isTriggerPoint = i > minIndexCountPerBucket && (i - 1) % minIndexCountPerBucket == 0; + if (isTriggerPoint) { + final int expectedBuckets = (i - 1) / minIndexCountPerBucket; + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .pollInterval(10, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + // 1. Confirm the number of immutable buckets matches the expected + assertEquals(tracker.getImmutableBuckets().asMapOfRanges().size(), expectedBuckets, + "Expected number of immutable buckets did not match."); + // 2. Confirm that the persistence Future for + // all snapshots of created immutable buckets have all completed + assertTrue(tracker.getImmutableBuckets().asMapOfRanges().values().stream() + .allMatch(bucket -> bucket.getSnapshotCreateFuture() + .map(CompletableFuture::isDone) + .orElse(true)), + "Not all snapshot creation futures were completed."); + }); + } } - assertEquals(tracker.getNumberOfDelayedMessages(), 100); - - clockTime.set(1 * 10); - - Awaitility.await().untilAsserted(() -> { - Assert.assertTrue( - tracker.getImmutableBuckets().asMapOfRanges().values().stream().noneMatch(x -> x.merging - || !x.getSnapshotCreateFuture().get().isDone())); - }); - - assertTrue(tracker.hasMessageAvailable()); - Set scheduledMessages = new TreeSet<>(); - Awaitility.await().untilAsserted(() -> { - scheduledMessages.addAll(tracker.getScheduledMessages(100)); - assertEquals(scheduledMessages.size(), 1); - }); - - tracker.addMessage(101, 101, 101 * 10); + assertEquals(tracker.getNumberOfDelayedMessages(), 101); + assertEquals(tracker.getImmutableBuckets().asMapOfRanges().size(), 20); + assertEquals(tracker.getLastMutableBucket().size(), 1); tracker.close(); @@ -306,7 +313,7 @@ public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) throws clockTime.set(110 * 10); NavigableSet scheduledMessages = new TreeSet<>(); - Awaitility.await().untilAsserted(() -> { + Awaitility.await().atMost(20, TimeUnit.SECONDS).untilAsserted(() -> { scheduledMessages.addAll(tracker2.getScheduledMessages(110)); assertEquals(scheduledMessages.size(), 110); }); @@ -430,11 +437,33 @@ public void testWithCreateFailDowngrade(BucketDelayedDeliveryTracker tracker) { @Test(dataProvider = "delayedTracker") public void testMaxIndexesPerSegment(BucketDelayedDeliveryTracker tracker) { - for (int i = 1; i <= 101; i++) { + final int minIndexCountPerBucket = 20; + final int totalMessages = 101; + final int expectedFinalBucketCount = (totalMessages - 1) / minIndexCountPerBucket; + for (int i = 1; i <= totalMessages; i++) { tracker.addMessage(i, i, i * 10); + + // The trigger point is the next message after the bucket is full. + // For example, i=21 triggers the 1st bucket, i=41 triggers the 2nd, ..., i=101 triggers the 5th + boolean isTriggerPoint = i > minIndexCountPerBucket && (i - 1) % minIndexCountPerBucket == 0; + + if (isTriggerPoint) { + final int expectedBuckets = (i - 1) / minIndexCountPerBucket; + + // Wait until the background bucket creation task is completed + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .pollInterval(10, TimeUnit.MILLISECONDS) + .untilAsserted(() -> + assertEquals(tracker.getImmutableBuckets().asMapOfRanges().size(), expectedBuckets) + ); + } } - assertEquals(tracker.getImmutableBuckets().asMapOfRanges().size(), 5); + // After the loop ends, we expect to have 5 immutable buckets + assertEquals(tracker.getImmutableBuckets().asMapOfRanges().size(), expectedFinalBucketCount); + + // And, the lastMutableBucket should only have the last message (101) left. + assertEquals(tracker.getLastMutableBucket().size(), 1); tracker.getImmutableBuckets().asMapOfRanges().forEach((k, bucket) -> { assertEquals(bucket.getLastSegmentEntryId(), 4); @@ -446,21 +475,23 @@ public void testMaxIndexesPerSegment(BucketDelayedDeliveryTracker tracker) { @Test(dataProvider = "delayedTracker") public void testClear(BucketDelayedDeliveryTracker tracker) throws ExecutionException, InterruptedException, TimeoutException { - for (int i = 1; i <= 1001; i++) { + for (int i = 1; i <= 1001; i++) { tracker.addMessage(i, i, i * 10); - } + } - assertEquals(tracker.getNumberOfDelayedMessages(), 1001); - assertTrue(tracker.getImmutableBuckets().asMapOfRanges().size() > 0); - assertEquals(tracker.getLastMutableBucket().size(), 1); + assertEquals(tracker.getNumberOfDelayedMessages(), 1001); + Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> + Assert.assertFalse(tracker.getImmutableBuckets().asMapOfRanges().isEmpty()) + ); + assertEquals(tracker.getLastMutableBucket().size(), 1); - tracker.clear().get(1, TimeUnit.MINUTES); + tracker.clear().get(1, TimeUnit.MINUTES); - assertEquals(tracker.getNumberOfDelayedMessages(), 0); - assertEquals(tracker.getImmutableBuckets().asMapOfRanges().size(), 0); - assertEquals(tracker.getLastMutableBucket().size(), 0); - assertEquals(tracker.getSharedBucketPriorityQueue().size(), 0); + assertEquals(tracker.getNumberOfDelayedMessages(), 0); + assertEquals(tracker.getImmutableBuckets().asMapOfRanges().size(), 0); + assertEquals(tracker.getLastMutableBucket().size(), 0); + assertEquals(tracker.getSharedBucketPriorityQueue().size(), 0); - tracker.close(); + tracker.close(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerThreadSafetyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerThreadSafetyTest.java index 3bc96499bfdef..20b76939a4b7c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerThreadSafetyTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerThreadSafetyTest.java @@ -27,17 +27,19 @@ import com.google.common.util.concurrent.MoreExecutors; import io.netty.util.Timer; import java.time.Clock; +import java.util.NavigableSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Phaser; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -45,14 +47,14 @@ /** * Thread safety tests for BucketDelayedDeliveryTracker. - * These tests verify that the hybrid approach with StampedLock and concurrent data structures + * These tests verify that the hybrid approach with ReentrantReadWriteLock and concurrent data structures * correctly handles concurrent access patterns without deadlocks, race conditions, or data corruption. */ +@Slf4j public class BucketDelayedDeliveryTrackerThreadSafetyTest { private BucketDelayedDeliveryTracker tracker; private AbstractPersistentDispatcherMultipleConsumers dispatcher; - private ManagedCursor cursor; private Timer timer; private BucketSnapshotStorage storage; private ExecutorService executorService; @@ -60,7 +62,7 @@ public class BucketDelayedDeliveryTrackerThreadSafetyTest { @BeforeMethod public void setUp() throws Exception { dispatcher = mock(AbstractPersistentDispatcherMultipleConsumers.class); - cursor = mock(ManagedCursor.class); + final ManagedCursor cursor = mock(ManagedCursor.class); timer = mock(Timer.class); storage = mock(BucketSnapshotStorage.class); @@ -97,88 +99,103 @@ public void setUp() throws Exception { @AfterMethod public void tearDown() throws Exception { - if (tracker != null) { - tracker.close(); - } + // First shutdown executor to stop all threads if (executorService != null) { assertTrue(MoreExecutors.shutdownAndAwaitTermination(executorService, 5, TimeUnit.SECONDS), - "Executor should shutdown cleanly"); + "Executor should shutdown cleanly"); + } + // Then close tracker safely after all threads stopped + if (tracker != null) { + tracker.close(); } } /** - * Test concurrent containsMessage() calls while adding messages. - * This tests the StampedLock optimistic read performance under contention. + * Test concurrent containsMessage() calls while adding messages sequentially. + * This tests the ReentrantReadWriteLock read performance under contention. + * addMessage is executed sequentially (as in real scenarios), while containsMessage is concurrent. */ @Test public void testConcurrentContainsMessageWithWrites() throws Exception { - final int numThreads = 16; - final int operationsPerThread = 1000; // Restore to test bucket creation properly + final int numReadThreads = 8; + final int readsPerThread = 1000; + final int totalMessages = 5000; final CountDownLatch startLatch = new CountDownLatch(1); - final CountDownLatch doneLatch = new CountDownLatch(numThreads); + final CountDownLatch readersDone = new CountDownLatch(numReadThreads); final AtomicInteger errors = new AtomicInteger(0); final AtomicReference firstException = new AtomicReference<>(); + final AtomicInteger messagesAdded = new AtomicInteger(0); - // Start reader threads - for (int i = 0; i < numThreads / 2; i++) { - final int threadId = i; + // Start reader threads - these will run concurrently + for (int i = 0; i < numReadThreads; i++) { executorService.submit(() -> { try { startLatch.await(); - for (int j = 0; j < operationsPerThread; j++) { - long ledgerId = threadId * 1000 + j; - long entryId = j; + // Continuously read for a period while messages are being added + long endTime = System.currentTimeMillis() + 10000; + int readCount = 0; + while (System.currentTimeMillis() < endTime && readCount < readsPerThread) { + // Check for messages across the range that might be added + long ledgerId = 1000 + (readCount % totalMessages); + long entryId = readCount % 100; // This should not throw exceptions or block indefinitely tracker.containsMessage(ledgerId, entryId); + readCount++; + if (readCount % 100 == 0) { + Thread.sleep(1); + } } } catch (Exception e) { errors.incrementAndGet(); firstException.compareAndSet(null, e); e.printStackTrace(); } finally { - doneLatch.countDown(); + readersDone.countDown(); } }); } - // Start writer threads - for (int i = numThreads / 2; i < numThreads; i++) { - final int threadId = i; - executorService.submit(() -> { - try { - startLatch.await(); - for (int j = 0; j < operationsPerThread; j++) { - long ledgerId = threadId * 1000 + j; - long entryId = j; - long deliverAt = System.currentTimeMillis() + 10000; // 10s delay - tracker.addMessage(ledgerId, entryId, deliverAt); + // Start the single writer thread - sequential addMessage calls + executorService.submit(() -> { + try { + startLatch.await(); + for (int i = 0; i < totalMessages; i++) { + long ledgerId = 1000 + i; + long entryId = i % 100; + long deliverAt = System.currentTimeMillis() + 10000; + boolean added = tracker.addMessage(ledgerId, entryId, deliverAt); + if (added) { + messagesAdded.incrementAndGet(); + } + // Small delay to simulate real processing time + if (i % 100 == 0) { + Thread.sleep(1); } - } catch (Exception e) { - errors.incrementAndGet(); - firstException.compareAndSet(null, e); - e.printStackTrace(); - } finally { - doneLatch.countDown(); } - }); - } + } catch (Exception e) { + errors.incrementAndGet(); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } + }); startLatch.countDown(); - assertTrue(doneLatch.await(30, TimeUnit.SECONDS), "Test should complete within 30 seconds"); + assertTrue(readersDone.await(30, TimeUnit.SECONDS), "Readers should complete within 30 seconds"); if (errors.get() > 0) { Exception exception = firstException.get(); if (exception != null) { - System.err.println("First exception caught: " + exception.getMessage()); + log.error("First exception caught: " + exception.getMessage()); exception.printStackTrace(); } } assertEquals(errors.get(), 0, "No exceptions should occur during concurrent operations"); + assertTrue(messagesAdded.get() > 0, "Some messages should have been added"); } /** * Test concurrent nextDeliveryTime() calls. - * This verifies the StampedLock implementation for read-heavy operations. + * This verifies the ReentrantReadWriteLock implementation for read-heavy operations. */ @Test public void testConcurrentNextDeliveryTime() throws Exception { @@ -224,75 +241,149 @@ public void testConcurrentNextDeliveryTime() throws Exception { */ @Test public void testDeadlockDetection() throws Exception { - final int numThreads = 32; - final int operationsPerThread = 100; - // Use Phaser for better concurrency coordination - final Phaser startPhaser = new Phaser(numThreads + 1); // +1 for main thread - final Phaser endPhaser = new Phaser(numThreads + 1); // +1 for main thread + final int numReadThreads = 30; + final int operationsPerThread = 200; + final int writeOperations = 1000; + + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch writerDone = new CountDownLatch(1); + final CountDownLatch readersDone = new CountDownLatch(numReadThreads); final AtomicBoolean deadlockDetected = new AtomicBoolean(false); final AtomicInteger completedOperations = new AtomicInteger(0); - - // Mixed workload: reads, writes, and metric queries - for (int i = 0; i < numThreads; i++) { + final AtomicInteger writesCompleted = new AtomicInteger(0); + final AtomicReference firstException = new AtomicReference<>(); + // Single writer thread - executes addMessage sequentially + executorService.submit(() -> { + try { + startLatch.await(); + for (int i = 0; i < writeOperations; i++) { + try { + long ledgerId = 50000 + i; + long entryId = i; + tracker.addMessage(ledgerId, entryId, System.currentTimeMillis() + 10000); + writesCompleted.incrementAndGet(); + completedOperations.incrementAndGet(); + + // Small delay to allow read threads to interleave + if (i % 50 == 0) { + Thread.sleep(1); + } + } catch (Exception e) { + if (!(e instanceof IllegalArgumentException)) { + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); + break; + } + completedOperations.incrementAndGet(); + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + deadlockDetected.set(true); + } catch (Exception e) { + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } finally { + writerDone.countDown(); + } + }); + // Multiple reader threads - execute read operations concurrently + for (int i = 0; i < numReadThreads; i++) { final int threadId = i; - final int workloadType = i % 4; - + final int readOperationType = i % 3; executorService.submit(() -> { try { - // Wait for all threads to be ready - startPhaser.arriveAndAwaitAdvance(); + startLatch.await(); - for (int j = 0; j < operationsPerThread; j++) { + // Continue reading until writer is done, plus some extra operations + int operationCount = 0; + while ((writerDone.getCount() > 0 || operationCount < operationsPerThread)) { try { - switch (workloadType) { - case 0: // containsMessage calls - tracker.containsMessage(threadId * 1000 + j, j); + switch (readOperationType) { + case 0: + // Check both existing and potentially non-existing messages + long ledgerId = 50000 + (operationCount % (writeOperations + 100)); + long entryId = operationCount % 1000; + tracker.containsMessage(ledgerId, entryId); break; - case 1: // addMessage calls - tracker.addMessage(threadId * 1000 + j, j, System.currentTimeMillis() + 5000); - break; - case 2: // nextDeliveryTime calls + case 1: tracker.nextDeliveryTime(); break; - case 3: // getNumberOfDelayedMessages calls + case 2: tracker.getNumberOfDelayedMessages(); break; } completedOperations.incrementAndGet(); + operationCount++; + + // Small delay to prevent excessive CPU usage + if (operationCount % 100 == 0) { + Thread.sleep(1); + } } catch (IllegalArgumentException e) { - // IllegalArgumentException is expected for some operations - // (e.g., calling nextDeliveryTime on empty queue, invalid ledger IDs) - // This is not a deadlock, just normal validation + // Expected for some operations (e.g., nextDeliveryTime on empty queue) completedOperations.incrementAndGet(); + operationCount++; + } catch (Exception e) { + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); + break; + } + + // Break if we've done enough operations and writer is done + if (writerDone.getCount() == 0 && operationCount >= operationsPerThread) { + break; } } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + deadlockDetected.set(true); } catch (Exception e) { - // Only unexpected exceptions indicate potential deadlocks - if (!(e instanceof IllegalArgumentException)) { - deadlockDetected.set(true); - e.printStackTrace(); - } + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); } finally { - // Signal completion - endPhaser.arriveAndDeregister(); + readersDone.countDown(); } }); } + // Start all threads + startLatch.countDown(); + // Wait for completion with timeout to detect deadlocks + boolean writerCompleted = false; + boolean readersCompleted = false; - // Start all threads at once - startPhaser.arriveAndAwaitAdvance(); - - // Wait for all threads to complete with timeout to detect potential deadlocks try { - endPhaser.awaitAdvanceInterruptibly(endPhaser.arrive(), 60, TimeUnit.SECONDS); - } catch (Exception e) { - // Timeout or interrupt indicates potential deadlock + writerCompleted = writerDone.await(30, TimeUnit.SECONDS); + readersCompleted = readersDone.await(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); deadlockDetected.set(true); - e.printStackTrace(); } - - assertTrue(!deadlockDetected.get(), "No deadlocks should be detected"); + // Check for deadlock indicators + if (!writerCompleted || !readersCompleted) { + deadlockDetected.set(true); + log.error("Test timed out - potential deadlock detected. Writer completed: {}, Readers completed: {}", + writerCompleted, readersCompleted); + } + // Assert results + if (deadlockDetected.get()) { + Exception e = firstException.get(); + if (e != null) { + throw new AssertionError("Deadlock or exception detected during test execution", e); + } else { + throw new AssertionError("Deadlock detected - test did not complete within timeout"); + } + } + // Verify that operations actually completed assertTrue(completedOperations.get() > 0, "Some operations should complete"); + assertTrue(writesCompleted.get() > 0, "Some write operations should complete"); + + log.info("Deadlock test completed successfully. Total operations: {}, Writes completed: {}", + completedOperations.get(), writesCompleted.get()); } /** @@ -301,85 +392,100 @@ public void testDeadlockDetection() throws Exception { */ @Test public void testDataConsistencyUnderConcurrency() throws Exception { - final int numWriteThreads = 8; final int numReadThreads = 16; - final int messagesPerWriter = 500; + final int totalMessages = 4000; + final int readsPerThread = 1000; final CountDownLatch startLatch = new CountDownLatch(1); - final CountDownLatch writersDone = new CountDownLatch(numWriteThreads); final CountDownLatch readersDone = new CountDownLatch(numReadThreads); + final AtomicInteger errors = new AtomicInteger(0); + final AtomicReference firstException = new AtomicReference<>(); final AtomicInteger foundMessages = new AtomicInteger(0); final AtomicInteger totalMessagesAdded = new AtomicInteger(0); - - // Writer threads add messages - for (int i = 0; i < numWriteThreads; i++) { - final int writerId = i; - executorService.submit(() -> { - try { - startLatch.await(); - for (int j = 0; j < messagesPerWriter; j++) { - long ledgerId = writerId * 10000 + j; - long entryId = j; - boolean added = tracker.addMessage(ledgerId, entryId, System.currentTimeMillis() + 30000); - if (added) { - totalMessagesAdded.incrementAndGet(); - } - } - } catch (Exception e) { - // Ignore exceptions for this test - } finally { - writersDone.countDown(); - } - }); - } - - // Reader threads check for messages + // Start reader threads - these will run concurrently for (int i = 0; i < numReadThreads; i++) { final int readerId = i; executorService.submit(() -> { try { startLatch.await(); + // Continuously read for a period while messages are being added + long endTime = System.currentTimeMillis() + 12000; + int readCount = 0; + while (System.currentTimeMillis() < endTime && readCount < readsPerThread) { + // Check for messages across the range that might be added + int messageIndex = readCount % totalMessages; + long ledgerId = 10000 + messageIndex; + long entryId = messageIndex % 100; + + if (tracker.containsMessage(ledgerId, entryId)) { + foundMessages.incrementAndGet(); + } + readCount++; - // Read for a while to catch messages being added - long endTime = System.currentTimeMillis() + 5000; // Read for 5 seconds - while (System.currentTimeMillis() < endTime) { - for (int writerId = 0; writerId < numWriteThreads; writerId++) { - for (int j = 0; j < messagesPerWriter; j++) { - long ledgerId = writerId * 10000 + j; - long entryId = j; - if (tracker.containsMessage(ledgerId, entryId)) { - foundMessages.incrementAndGet(); - } - } + if (readCount % 200 == 0) { + Thread.sleep(1); } - Thread.sleep(10); // Small delay to allow writes } } catch (Exception e) { - // Ignore exceptions for this test + errors.incrementAndGet(); + firstException.compareAndSet(null, e); + e.printStackTrace(); } finally { readersDone.countDown(); } }); } - + // Start the single writer thread - sequential addMessage calls + executorService.submit(() -> { + try { + startLatch.await(); + for (int i = 0; i < totalMessages; i++) { + long ledgerId = 10000 + i; + long entryId = i % 100; + long deliverAt = System.currentTimeMillis() + 30000; + boolean added = tracker.addMessage(ledgerId, entryId, deliverAt); + if (added) { + totalMessagesAdded.incrementAndGet(); + } + // Small delay to simulate real processing time and allow reads + if (i % 200 == 0) { + Thread.sleep(2); + } + } + } catch (Exception e) { + errors.incrementAndGet(); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } + }); startLatch.countDown(); - - assertTrue(writersDone.await(30, TimeUnit.SECONDS), "Writers should complete"); - assertTrue(readersDone.await(30, TimeUnit.SECONDS), "Readers should complete"); - + assertTrue(readersDone.await(40, TimeUnit.SECONDS), "Readers should complete within 40 seconds"); + // Check for errors during concurrent operations + if (errors.get() > 0) { + Exception exception = firstException.get(); + if (exception != null) { + log.error("First exception caught: " + exception.getMessage()); + exception.printStackTrace(); + } + } + assertEquals(errors.get(), 0, "No exceptions should occur during concurrent operations"); // Verify final consistency long finalMessageCount = tracker.getNumberOfDelayedMessages(); assertTrue(finalMessageCount >= 0, "Message count should be non-negative"); - - // The exact counts may vary due to timing, but we should have some successful operations + // The exact counts may vary due to timing, but we should have successful operations assertTrue(totalMessagesAdded.get() > 0, "Some messages should have been added"); + assertTrue(foundMessages.get() >= 0, "Found messages count should be non-negative"); + + // Log results for analysis + log.info("Total messages added: {}, Found messages: {}, Final message count: {}", + totalMessagesAdded.get(), foundMessages.get(), finalMessageCount); } /** - * Test optimistic read performance under varying contention levels. - * This helps validate that the StampedLock optimistic reads are working efficiently. + * Test read performance under varying contention levels. + * This helps validate that the ReentrantReadWriteLock reads are working efficiently. */ @Test - public void testOptimisticReadPerformance() throws Exception { + public void testReadPerformanceUnderContention() throws Exception { // Add baseline messages for (int i = 0; i < 1000; i++) { tracker.addMessage(i, i, System.currentTimeMillis() + 60000); @@ -430,4 +536,483 @@ public void testOptimisticReadPerformance() throws Exception { assertTrue(throughput > 10000, "Should achieve at least 10K ops/sec with " + numThreads + " threads"); } } + + /** + * Test concurrent getScheduledMessages() calls with read operations. + * getScheduledMessages() uses write lock while read operations use read lock. + * Messages are added beforehand to avoid concurrent addMessage calls. + */ + @Test + public void testConcurrentGetScheduledMessagesWithReads() throws Exception { + // Add messages that will be ready for delivery after a short delay + final long baseTime = System.currentTimeMillis(); + final int totalMessages = 500; + + // Add messages with delivery time slightly in the future, then wait for them to become ready + for (int i = 0; i < totalMessages; i++) { + tracker.addMessage(i, i, baseTime + 1000); + } + assertEquals(tracker.getNumberOfDelayedMessages(), totalMessages, "All messages should be added"); + // Wait for messages to become ready for delivery + Thread.sleep(3000); + final int numReadThreads = 12; + final int numScheduleThreads = 4; + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch readersDone = new CountDownLatch(numReadThreads); + final CountDownLatch schedulersDone = new CountDownLatch(numScheduleThreads); + final AtomicInteger errors = new AtomicInteger(0); + final AtomicReference firstException = new AtomicReference<>(); + final AtomicInteger totalMessagesRetrieved = new AtomicInteger(0); + // Start read threads (containsMessage and nextDeliveryTime) + for (int i = 0; i < numReadThreads; i++) { + final int threadId = i; + executorService.submit(() -> { + try { + startLatch.await(); + for (int j = 0; j < 1000; j++) { + if (threadId % 2 == 0) { + tracker.containsMessage(j % totalMessages, j % totalMessages); + } else { + try { + tracker.nextDeliveryTime(); + } catch (IllegalArgumentException e) { + // Expected when no messages available + } + } + if (j % 100 == 0) { + Thread.sleep(1); + } + } + } catch (Exception e) { + errors.incrementAndGet(); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } finally { + readersDone.countDown(); + } + }); + } + // Start getScheduledMessages threads - continue until all messages are retrieved + for (int i = 0; i < numScheduleThreads; i++) { + executorService.submit(() -> { + try { + startLatch.await(); + int consecutiveEmptyReturns = 0; + final int maxConsecutiveEmpty = 5; + + while (totalMessagesRetrieved.get() < totalMessages + && consecutiveEmptyReturns < maxConsecutiveEmpty) { + NavigableSet messages = tracker.getScheduledMessages(50); + int retrieved = messages.size(); + totalMessagesRetrieved.addAndGet(retrieved); + + if (retrieved == 0) { + consecutiveEmptyReturns++; + Thread.sleep(10); + } else { + consecutiveEmptyReturns = 0; + Thread.sleep(5); + } + } + } catch (Exception e) { + errors.incrementAndGet(); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } finally { + schedulersDone.countDown(); + } + }); + } + startLatch.countDown(); + assertTrue(readersDone.await(30, TimeUnit.SECONDS), "Readers should complete within 30 seconds"); + assertTrue(schedulersDone.await(30, TimeUnit.SECONDS), "Schedulers should complete within 30 seconds"); + if (errors.get() > 0) { + Exception exception = firstException.get(); + if (exception != null) { + throw new AssertionError("Concurrent getScheduledMessages test failed", exception); + } + } + assertEquals(errors.get(), 0, "No exceptions should occur during concurrent operations"); + + // Verify that most or all messages were retrieved + assertEquals(totalMessagesRetrieved.get(), 500, "All messages should be retrieved"); + + log.info("Total messages retrieved: {} out of {}", totalMessagesRetrieved.get(), totalMessages); + } + + /** + * Test concurrent clear() operations with read operations. + * This verifies that clear() properly coordinates with ongoing read operations. + * Messages are added beforehand, then clear() is tested with concurrent reads. + */ + @Test + public void testConcurrentClearWithReads() throws Exception { + final int initialMessages = 1000; + final int numReadThreads = 10; + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch readersDone = new CountDownLatch(numReadThreads); + final AtomicInteger errors = new AtomicInteger(0); + final AtomicReference firstException = new AtomicReference<>(); + final AtomicBoolean clearCompleted = new AtomicBoolean(false); + // Add initial messages (single thread) + for (int i = 0; i < initialMessages; i++) { + tracker.addMessage(i, i, System.currentTimeMillis() + 60000); + } + // Start read threads that will run during clear operation + for (int i = 0; i < numReadThreads; i++) { + final int threadId = i; + executorService.submit(() -> { + try { + startLatch.await(); + while (!clearCompleted.get()) { + switch (threadId % 3) { + case 0: + tracker.containsMessage(threadId, threadId); + break; + case 1: + try { + tracker.nextDeliveryTime(); + } catch (IllegalArgumentException e) { + // Expected when no messages available + } + break; + case 2: + tracker.getNumberOfDelayedMessages(); + break; + } + Thread.sleep(1); + } + // Continue reading for a bit after clear + for (int j = 0; j < 100; j++) { + tracker.containsMessage(j, j); + Thread.sleep(1); + } + } catch (Exception e) { + errors.incrementAndGet(); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } finally { + readersDone.countDown(); + } + }); + } + // Start clear operation after a short delay + executorService.submit(() -> { + try { + startLatch.await(); + Thread.sleep(100); + tracker.clear().get(30, TimeUnit.SECONDS); + clearCompleted.set(true); + } catch (Exception e) { + errors.incrementAndGet(); + firstException.compareAndSet(null, e); + e.printStackTrace(); + clearCompleted.set(true); + } + }); + startLatch.countDown(); + assertTrue(readersDone.await(60, TimeUnit.SECONDS), "Readers should complete within 60 seconds"); + if (errors.get() > 0) { + Exception exception = firstException.get(); + if (exception != null) { + throw new AssertionError("Concurrent clear test failed", exception); + } + } + assertEquals(errors.get(), 0, "No exceptions should occur during concurrent clear operations"); + assertEquals(tracker.getNumberOfDelayedMessages(), 0, "All messages should be cleared"); + } + + /** + * Test concurrent close() operations. + * This verifies that close() properly handles concurrent access and shuts down cleanly. + * Messages are added beforehand to test close() behavior with existing data. + */ + @Test + public void testConcurrentClose() throws Exception { + final int numReadThreads = 8; + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch readersDone = new CountDownLatch(numReadThreads); + final AtomicInteger errors = new AtomicInteger(0); + final AtomicReference firstException = new AtomicReference<>(); + final AtomicBoolean closeInitiated = new AtomicBoolean(false); + // Add some messages first (single thread) + for (int i = 0; i < 100; i++) { + tracker.addMessage(i, i, System.currentTimeMillis() + 60000); + } + // Start read threads that will be interrupted by close + for (int i = 0; i < numReadThreads; i++) { + final int threadId = i; + executorService.submit(() -> { + try { + startLatch.await(); + while (!closeInitiated.get()) { + try { + switch (threadId % 4) { + case 0: + tracker.containsMessage(threadId, threadId); + break; + case 1: + tracker.nextDeliveryTime(); + break; + case 2: + tracker.getNumberOfDelayedMessages(); + break; + case 3: + tracker.getScheduledMessages(10); + break; + } + } catch (IllegalArgumentException e) { + // Expected for some operations when tracker is being closed + } + Thread.sleep(1); + } + } catch (Exception e) { + // Some exceptions may be expected during close + if (!closeInitiated.get()) { + errors.incrementAndGet(); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } + } finally { + readersDone.countDown(); + } + }); + } + // Start close operation after a short delay + executorService.submit(() -> { + try { + startLatch.await(); + Thread.sleep(100); + closeInitiated.set(true); + tracker.close(); + } catch (Exception e) { + errors.incrementAndGet(); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } + }); + startLatch.countDown(); + assertTrue(readersDone.await(30, TimeUnit.SECONDS), "Readers should complete within 30 seconds"); + if (errors.get() > 0) { + Exception exception = firstException.get(); + if (exception != null) { + log.warn("Exception during concurrent close test (may be expected): " + exception.getMessage()); + } + } + // Create a new tracker for the next test since this one is closed + tracker = new BucketDelayedDeliveryTracker( + dispatcher, timer, 1000, Clock.systemUTC(), true, storage, + 100, 1000, 100, 10 + ); + } + + /** + * Test mixed read operations with sequential addMessage and concurrent getScheduledMessages. + * This tests the ReentrantReadWriteLock behavior when read and write operations are mixed. + * addMessage is executed in single thread, while reads and getScheduledMessages are concurrent. + * Ensures all deliverable messages are retrieved before test completion. + */ + @Test + public void testMixedReadWriteOperationsDeadlockDetection() throws Exception { + final int numReadThreads = 16; + final int numScheduleThreads = 4; + final int totalMessages = 2000; + final int readsPerThread = 500; + + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch readersDone = new CountDownLatch(numReadThreads); + final CountDownLatch schedulersDone = new CountDownLatch(numScheduleThreads); + final CountDownLatch writerDone = new CountDownLatch(1); + final AtomicBoolean deadlockDetected = new AtomicBoolean(false); + final AtomicInteger completedOperations = new AtomicInteger(0); + final AtomicReference firstException = new AtomicReference<>(); + final AtomicInteger messagesAdded = new AtomicInteger(0); + final AtomicInteger deliverableMessagesCount = new AtomicInteger(0); + final AtomicInteger totalMessagesRetrieved = new AtomicInteger(0); + // Single writer thread for addMessage (sequential execution) + executorService.submit(() -> { + try { + startLatch.await(); + final long baseTime = System.currentTimeMillis(); + + for (int i = 0; i < totalMessages; i++) { + try { + long ledgerId = 10000 + i; + long entryId = i % 1000; + + // Create mix of messages: some ready for delivery, some delayed + long deliverAt; + if (i % 3 == 0) { + // Messages that will be ready for delivery after a short delay + deliverAt = baseTime + 500; + deliverableMessagesCount.incrementAndGet(); + } else { + // Messages for future delivery (much later) + deliverAt = baseTime + 30000; + } + + boolean added = tracker.addMessage(ledgerId, entryId, deliverAt); + if (added) { + messagesAdded.incrementAndGet(); + } + completedOperations.incrementAndGet(); + + if (i % 200 == 0) { + Thread.sleep(1); + } + } catch (Exception e) { + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); + break; + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + deadlockDetected.set(true); + } catch (Exception e) { + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } finally { + writerDone.countDown(); + } + }); + // Start read threads (using read locks) + for (int i = 0; i < numReadThreads; i++) { + final int threadId = i; + executorService.submit(() -> { + try { + startLatch.await(); + // Continue reading until writer is done, plus some extra operations + int operationCount = 0; + while ((writerDone.getCount() > 0 || operationCount < readsPerThread)) { + try { + switch (threadId % 3) { + case 0: + long ledgerId = 10000 + (operationCount % (totalMessages + 100)); + long entryId = operationCount % 1000; + tracker.containsMessage(ledgerId, entryId); + break; + case 1: + tracker.nextDeliveryTime(); + break; + case 2: + tracker.getNumberOfDelayedMessages(); + break; + } + completedOperations.incrementAndGet(); + operationCount++; + if (operationCount % 100 == 0) { + Thread.sleep(1); + } + } catch (IllegalArgumentException e) { + // Expected for some operations + completedOperations.incrementAndGet(); + operationCount++; + } catch (Exception e) { + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); + break; + } + if (writerDone.getCount() == 0 && operationCount >= readsPerThread) { + break; + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + deadlockDetected.set(true); + } catch (Exception e) { + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } finally { + readersDone.countDown(); + } + }); + } + // Start getScheduledMessages threads (using write locks) + for (int i = 0; i < numScheduleThreads; i++) { + executorService.submit(() -> { + try { + startLatch.await(); + + // Wait for writer to finish and messages to become deliverable + writerDone.await(); + Thread.sleep(1000); // Wait 1 second for messages to become ready for delivery + + int consecutiveEmptyReturns = 0; + final int maxConsecutiveEmpty = 5; + + // Continue until we've retrieved all deliverable messages or hit max empty returns + while (totalMessagesRetrieved.get() < deliverableMessagesCount.get() + && consecutiveEmptyReturns < maxConsecutiveEmpty) { + try { + NavigableSet messages = tracker.getScheduledMessages(50); + int retrieved = messages.size(); + totalMessagesRetrieved.addAndGet(retrieved); + completedOperations.incrementAndGet(); + + if (retrieved == 0) { + consecutiveEmptyReturns++; + Thread.sleep(5); // Short wait for more messages + } else { + consecutiveEmptyReturns = 0; + Thread.sleep(2); // Short processing delay + } + + } catch (Exception e) { + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); + break; + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + deadlockDetected.set(true); + } catch (Exception e) { + deadlockDetected.set(true); + firstException.compareAndSet(null, e); + e.printStackTrace(); + } finally { + schedulersDone.countDown(); + } + }); + } + // Start all threads + startLatch.countDown(); + + // Wait for completion with reasonable timeout to detect deadlocks + boolean writerCompleted = writerDone.await(10, TimeUnit.SECONDS); + boolean readersCompleted = readersDone.await(15, TimeUnit.SECONDS); + boolean schedulersCompleted = schedulersDone.await(20, TimeUnit.SECONDS); + if (!writerCompleted || !readersCompleted || !schedulersCompleted) { + deadlockDetected.set(true); + log.error("Test timed out - potential deadlock detected. Writer: {}, Readers: {}, Schedulers: {}", + writerCompleted, readersCompleted, schedulersCompleted); + } + if (deadlockDetected.get()) { + Exception e = firstException.get(); + if (e != null) { + throw new AssertionError("Deadlock or exception detected during mixed operations test", e); + } else { + throw new AssertionError("Deadlock detected - test did not complete within timeout"); + } + } + + // Verify operations completed successfully + assertTrue(completedOperations.get() > 0, "Some operations should complete"); + assertTrue(messagesAdded.get() > 0, "Some messages should have been added"); + assertTrue(deliverableMessagesCount.get() > 0, "Some messages should be deliverable"); + + // Verify that all deliverable messages were retrieved + assertEquals(totalMessagesRetrieved.get(), deliverableMessagesCount.get(), + "All deliverable messages should be retrieved"); + log.info("Mixed operations test completed successfully. Total operations: {}, Messages added: {}, " + + "Deliverable messages: {}, Retrieved messages: {}", + completedOperations.get(), messagesAdded.get(), + deliverableMessagesCount.get(), totalMessagesRetrieved.get()); + } } \ No newline at end of file