Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[improve][broker] Cache LedgerHandle in BookkeeperBucketSnapshotStorage #20117

Merged
merged 8 commits into from
Apr 20, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import javax.validation.constraints.NotNull;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.client.BKException;
Expand All @@ -48,6 +49,8 @@ public class BookkeeperBucketSnapshotStorage implements BucketSnapshotStorage {
private final ServiceConfiguration config;
private BookKeeper bookKeeper;

private final Map<Long, CompletableFuture<LedgerHandle>> ledgerHandleFutureCache = new ConcurrentHashMap<>();

public BookkeeperBucketSnapshotStorage(PulsarService pulsar) {
this.pulsar = pulsar;
this.config = pulsar.getConfig();
Expand All @@ -66,45 +69,30 @@ public CompletableFuture<Long> createBucketSnapshot(SnapshotMetadata snapshotMet

@Override
public CompletableFuture<SnapshotMetadata> getBucketSnapshotMetadata(long bucketId) {
return openLedger(bucketId).thenCompose(ledgerHandle -> {
CompletableFuture<SnapshotMetadata> snapshotFuture =
getLedgerEntry(ledgerHandle, 0, 0)
.thenApply(entryEnumeration -> parseSnapshotMetadataEntry(entryEnumeration.nextElement()));

snapshotFuture.whenComplete((__, e) -> closeLedger(ledgerHandle));

return snapshotFuture;
});
return getLedgerHandle(bucketId).thenCompose(ledgerHandle -> getLedgerEntry(ledgerHandle, 0, 0)
.thenApply(entryEnumeration -> parseSnapshotMetadataEntry(entryEnumeration.nextElement())));
}

@Override
public CompletableFuture<List<SnapshotSegment>> getBucketSnapshotSegment(long bucketId, long firstSegmentEntryId,
long lastSegmentEntryId) {
return openLedger(bucketId).thenCompose(ledgerHandle -> {
CompletableFuture<List<SnapshotSegment>> parseFuture =
getLedgerEntry(ledgerHandle, firstSegmentEntryId, lastSegmentEntryId)
.thenApply(this::parseSnapshotSegmentEntries);

parseFuture.whenComplete((__, e) -> closeLedger(ledgerHandle));

return parseFuture;
});
return getLedgerHandle(bucketId).thenCompose(
ledgerHandle -> getLedgerEntry(ledgerHandle, firstSegmentEntryId, lastSegmentEntryId)
.thenApply(this::parseSnapshotSegmentEntries));
}

@Override
public CompletableFuture<Long> getBucketSnapshotLength(long bucketId) {
return openLedger(bucketId).thenCompose(ledgerHandle -> {
CompletableFuture<Long> lengthFuture =
CompletableFuture.completedFuture(ledgerHandle.getLength());

lengthFuture.whenComplete((__, e) -> closeLedger(ledgerHandle));

return lengthFuture;
});
return getLedgerHandle(bucketId).thenCompose(
ledgerHandle -> CompletableFuture.completedFuture(ledgerHandle.getLength()));
}

@Override
public CompletableFuture<Void> deleteBucketSnapshot(long bucketId) {
CompletableFuture<LedgerHandle> ledgerHandleFuture = ledgerHandleFutureCache.remove(bucketId);
if (ledgerHandleFuture != null) {
ledgerHandleFuture.whenComplete((lh, ex) -> closeLedger(lh));
}
return deleteLedger(bucketId);
}

Expand Down Expand Up @@ -178,6 +166,18 @@ private CompletableFuture<LedgerHandle> createLedger(String bucketKey, String to
return future;
}

private CompletableFuture<LedgerHandle> getLedgerHandle(Long ledgerId) {
CompletableFuture<LedgerHandle> ledgerHandleCompletableFuture =
ledgerHandleFutureCache.computeIfAbsent(ledgerId, k -> openLedger(ledgerId));
// remove future of completed exceptionally
ledgerHandleCompletableFuture.whenComplete((__, ex) -> {
if (ex != null) {
ledgerHandleFutureCache.remove(ledgerId, ledgerHandleCompletableFuture);
}
});
return ledgerHandleCompletableFuture;
}

private CompletableFuture<LedgerHandle> openLedger(Long ledgerId) {
final CompletableFuture<LedgerHandle> future = new CompletableFuture<>();
bookKeeper.asyncOpenLedger(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.broker.delayed.bucket.BookkeeperBucketSnapshotStorage;
import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat;
import org.apache.pulsar.common.util.FutureUtil;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
Expand Down Expand Up @@ -204,4 +205,46 @@ public void testGetBucketSnapshotLength() throws ExecutionException, Interrupted
Assert.assertTrue(bucketSnapshotLength > 0L);
}

@Test
public void testConcurrencyGet() throws ExecutionException, InterruptedException {
DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata segmentMetadata =
DelayedMessageIndexBucketSnapshotFormat.SnapshotSegmentMetadata.newBuilder()
.setMinScheduleTimestamp(System.currentTimeMillis())
.setMaxScheduleTimestamp(System.currentTimeMillis())
.putDelayedIndexBitMap(100L, ByteString.copyFrom(new byte[1])).build();

DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata snapshotMetadata =
DelayedMessageIndexBucketSnapshotFormat.SnapshotMetadata.newBuilder()
.addMetadataList(segmentMetadata)
.build();
List<DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment> bucketSnapshotSegments = new ArrayList<>();

long timeMillis = System.currentTimeMillis();
DelayedMessageIndexBucketSnapshotFormat.DelayedIndex delayedIndex =
DelayedMessageIndexBucketSnapshotFormat.DelayedIndex.newBuilder().setLedgerId(100L).setEntryId(10L)
.setTimestamp(timeMillis).build();
DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment snapshotSegment =
DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment.newBuilder().addIndexes(delayedIndex).build();
bucketSnapshotSegments.add(snapshotSegment);
bucketSnapshotSegments.add(snapshotSegment);

CompletableFuture<Long> future =
bucketSnapshotStorage.createBucketSnapshot(snapshotMetadata,
bucketSnapshotSegments, UUID.randomUUID().toString(), TOPIC_NAME, CURSOR_NAME);
Long bucketId = future.get();
Assert.assertNotNull(bucketId);

List<CompletableFuture<Void>> futures = new ArrayList<>();
for (int i = 0; i < 100; i++) {
CompletableFuture<Void> future0 = CompletableFuture.runAsync(() -> {
List<DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment> list =
bucketSnapshotStorage.getBucketSnapshotSegment(bucketId, 1, 3).join();
Assert.assertTrue(list.size() > 0);
});
futures.add(future0);
}

FutureUtil.waitForAll(futures).join();
}

}