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

[feat][broker][PIP-195] Implement delayed message index bucket snapshot(merge/delete) - part8 #19138

Merged
merged 7 commits into from
Jan 17, 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 @@ -143,4 +143,9 @@ private CompletableFuture<Void> putBucketKeyId(String bucketKey, Long bucketId)
return executeWithRetry(() -> cursor.putCursorProperty(bucketKey, String.valueOf(bucketId)),
ManagedLedgerException.BadVersionException.class, MaxRetryTimes);
}

protected CompletableFuture<Void> removeBucketCursorProperty(String bucketKey) {
return executeWithRetry(() -> cursor.removeCursorProperty(bucketKey),
ManagedLedgerException.BadVersionException.class, MaxRetryTimes);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.pulsar.broker.delayed.bucket.Bucket.DELAYED_BUCKET_KEY_PREFIX;
import static org.apache.pulsar.broker.delayed.bucket.Bucket.DELIMITER;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.HashBasedTable;
import com.google.common.collect.Range;
import com.google.common.collect.RangeMap;
Expand All @@ -32,14 +33,17 @@
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Optional;
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import javax.annotation.concurrent.ThreadSafe;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
Expand Down Expand Up @@ -71,6 +75,8 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker

private final TripleLongPriorityQueue sharedBucketPriorityQueue;

@Getter
@VisibleForTesting
private final RangeMap<Long, ImmutableBucket> immutableBuckets;

private final Table<Long, Long, ImmutableBucket> snapshotSegmentLastIndexTable;
Expand Down Expand Up @@ -105,15 +111,16 @@ public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispat

private synchronized long recoverBucketSnapshot() throws RuntimeException {
ManagedCursor cursor = this.lastMutableBucket.cursor;
Map<Range<Long>, ImmutableBucket> toBeDeletedBucketMap = new ConcurrentHashMap<>();
cursor.getCursorProperties().keySet().forEach(key -> {
if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) {
String[] keys = key.split(DELIMITER);
checkArgument(keys.length == 3);
ImmutableBucket immutableBucket =
new ImmutableBucket(cursor, this.lastMutableBucket.bucketSnapshotStorage,
Long.parseLong(keys[1]), Long.parseLong(keys[2]));
immutableBuckets.put(Range.closed(immutableBucket.startLedgerId, immutableBucket.endLedgerId),
immutableBucket);
putAndCleanOverlapRange(Range.closed(immutableBucket.startLedgerId, immutableBucket.endLedgerId),
immutableBucket, toBeDeletedBucketMap);
}
});

Expand All @@ -122,10 +129,14 @@ private synchronized long recoverBucketSnapshot() throws RuntimeException {
}

List<CompletableFuture<Void>> futures = new ArrayList<>(immutableBuckets.asMapOfRanges().size());
for (ImmutableBucket immutableBucket : immutableBuckets.asMapOfRanges().values()) {
for (Map.Entry<Range<Long>, ImmutableBucket> entry :immutableBuckets.asMapOfRanges().entrySet()) {
Range<Long> key = entry.getKey();
ImmutableBucket immutableBucket = entry.getValue();
CompletableFuture<Void> future =
immutableBucket.asyncRecoverBucketSnapshotEntry(this::getCutoffTime).thenAccept(indexList -> {
if (CollectionUtils.isEmpty(indexList)) {
// Delete bucket snapshot if indexList is empty
toBeDeletedBucketMap.put(key, immutableBucket);
return;
}
DelayedIndex lastDelayedIndex = indexList.get(indexList.size() - 1);
Expand All @@ -144,7 +155,12 @@ private synchronized long recoverBucketSnapshot() throws RuntimeException {
}

try {
FutureUtil.waitForAll(futures).get(AsyncOperationTimeoutSeconds, TimeUnit.SECONDS);
FutureUtil.waitForAll(futures).whenComplete((__, ex) -> {
toBeDeletedBucketMap.forEach((k, immutableBucket) -> {
immutableBuckets.asMapOfRanges().remove(k);
immutableBucket.asyncDeleteBucketSnapshot();
});
}).get(AsyncOperationTimeoutSeconds, TimeUnit.SECONDS);
} catch (InterruptedException | ExecutionException | TimeoutException e) {
throw new RuntimeException(e);
}
Expand All @@ -160,6 +176,26 @@ private synchronized long recoverBucketSnapshot() throws RuntimeException {
return numberDelayedMessages.getValue();
}

private synchronized void putAndCleanOverlapRange(Range<Long> range, ImmutableBucket immutableBucket,
Map<Range<Long>, ImmutableBucket> toBeDeletedBucketMap) {
RangeMap<Long, ImmutableBucket> subRangeMap = immutableBuckets.subRangeMap(range);
boolean canPut = false;
if (!subRangeMap.asMapOfRanges().isEmpty()) {
for (Map.Entry<Range<Long>, ImmutableBucket> rangeEntry : subRangeMap.asMapOfRanges().entrySet()) {
if (range.encloses(rangeEntry.getKey())) {
toBeDeletedBucketMap.put(rangeEntry.getKey(), rangeEntry.getValue());
canPut = true;
}
}
} else {
canPut = true;
}

if (canPut) {
immutableBuckets.put(range, immutableBucket);
}
}

@Override
public void run(Timeout timeout) throws Exception {
synchronized (this) {
Expand All @@ -179,10 +215,7 @@ private Optional<ImmutableBucket> findImmutableBucket(long ledgerId) {
return Optional.ofNullable(immutableBuckets.get(ledgerId));
}

private void sealBucket() {
Pair<ImmutableBucket, DelayedIndex> immutableBucketDelayedIndexPair =
lastMutableBucket.sealBucketAndAsyncPersistent(this.timeStepPerBucketSnapshotSegment,
this.sharedBucketPriorityQueue);
private void afterCreateImmutableBucket(Pair<ImmutableBucket, DelayedIndex> immutableBucketDelayedIndexPair) {
if (immutableBucketDelayedIndexPair != null) {
ImmutableBucket immutableBucket = immutableBucketDelayedIndexPair.getLeft();
immutableBuckets.put(Range.closed(immutableBucket.startLedgerId, immutableBucket.endLedgerId),
Expand Down Expand Up @@ -214,11 +247,21 @@ public synchronized boolean addMessage(long ledgerId, long entryId, long deliver
if (!existBucket && ledgerId > lastMutableBucket.endLedgerId
&& lastMutableBucket.size() >= minIndexCountPerBucket
&& !lastMutableBucket.isEmpty()) {
sealBucket();
Pair<ImmutableBucket, DelayedIndex> immutableBucketDelayedIndexPair =
lastMutableBucket.sealBucketAndAsyncPersistent(this.timeStepPerBucketSnapshotSegment,
this.sharedBucketPriorityQueue);
afterCreateImmutableBucket(immutableBucketDelayedIndexPair);
lastMutableBucket.resetLastMutableBucketRange();

if (immutableBuckets.asMapOfRanges().size() > maxNumBuckets) {
// TODO merge bucket snapshot (synchronize operate)
try {
asyncMergeBucketSnapshot().get(AsyncOperationTimeoutSeconds, TimeUnit.SECONDS);
} catch (InterruptedException | ExecutionException | TimeoutException e) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please call Thread.currentThread().interrupt(); if we get InterruptedException.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @mattisonchao
I wonder why we should keep the interrupted state. Is this state to be handed over to netty?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe not. But I think we should persist in this state if we don't have any special purpose.

if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw new RuntimeException(e);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you think we should consider the specific exception?

}
}
}

Expand All @@ -243,6 +286,55 @@ public synchronized boolean addMessage(long ledgerId, long entryId, long deliver
return true;
}

private synchronized CompletableFuture<Void> asyncMergeBucketSnapshot() {
List<ImmutableBucket> values = immutableBuckets.asMapOfRanges().values().stream().toList();
long minNumberMessages = Long.MAX_VALUE;
int minIndex = -1;
for (int i = 0; i + 1 < values.size(); i++) {
ImmutableBucket bucketL = values.get(i);
ImmutableBucket bucketR = values.get(i + 1);
long numberMessages = bucketL.numberBucketDelayedMessages + bucketR.numberBucketDelayedMessages;
if (numberMessages < minNumberMessages) {
minNumberMessages = (int) numberMessages;
minIndex = i;
}
}
return asyncMergeBucketSnapshot(values.get(minIndex), values.get(minIndex + 1));
}

private synchronized CompletableFuture<Void> asyncMergeBucketSnapshot(ImmutableBucket bucketA,
ImmutableBucket bucketB) {
immutableBuckets.remove(Range.closed(bucketA.startLedgerId, bucketA.endLedgerId));
immutableBuckets.remove(Range.closed(bucketB.startLedgerId, bucketB.endLedgerId));

CompletableFuture<Long> snapshotCreateFutureA =
bucketA.getSnapshotCreateFuture().orElse(CompletableFuture.completedFuture(null));
CompletableFuture<Long> snapshotCreateFutureB =
bucketB.getSnapshotCreateFuture().orElse(CompletableFuture.completedFuture(null));

return CompletableFuture.allOf(snapshotCreateFutureA, snapshotCreateFutureB).thenCompose(__ -> {
CompletableFuture<List<DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment>> futureA =
bucketA.getRemainSnapshotSegment();
CompletableFuture<List<DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment>> futureB =
bucketB.getRemainSnapshotSegment();
return futureA.thenCombine(futureB, CombinedSegmentDelayedIndexQueue::wrap)
.thenAccept(combinedDelayedIndexQueue -> {
Pair<ImmutableBucket, DelayedIndex> immutableBucketDelayedIndexPair =
lastMutableBucket.createImmutableBucketAndAsyncPersistent(
timeStepPerBucketSnapshotSegment, sharedBucketPriorityQueue,
combinedDelayedIndexQueue, bucketA.startLedgerId, bucketB.endLedgerId);
afterCreateImmutableBucket(immutableBucketDelayedIndexPair);

immutableBucketDelayedIndexPair.getLeft().getSnapshotCreateFuture()
.orElse(CompletableFuture.completedFuture(null)).thenCompose(___ -> {
CompletableFuture<Void> removeAFuture = bucketA.asyncDeleteBucketSnapshot();
CompletableFuture<Void> removeBFuture = bucketB.asyncDeleteBucketSnapshot();
return CompletableFuture.allOf(removeAFuture, removeBFuture);
});
});
});
}

@Override
public synchronized boolean hasMessageAvailable() {
long cutoffTime = getCutoffTime();
Expand Down Expand Up @@ -299,7 +391,7 @@ public synchronized NavigableSet<PositionImpl> getScheduledMessages(int maxMessa
removeIndexBit(ledgerId, entryId);

ImmutableBucket bucket = snapshotSegmentLastIndexTable.remove(ledgerId, entryId);
if (bucket != null) {
if (bucket != null && immutableBuckets.asMapOfRanges().containsValue(bucket)) {
if (log.isDebugEnabled()) {
log.debug("[{}] Load next snapshot segment, bucket: {}", dispatcher.getName(), bucket);
}
Expand All @@ -308,6 +400,8 @@ public synchronized NavigableSet<PositionImpl> getScheduledMessages(int maxMessa
try {
bucket.asyncLoadNextBucketSnapshotEntry().thenAccept(indexList -> {
if (CollectionUtils.isEmpty(indexList)) {
immutableBuckets.remove(Range.closed(bucket.startLedgerId, bucket.endLedgerId));
bucket.asyncDeleteBucketSnapshot();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need a log to show the exception.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add exception log in asyncDeleteBucketSnapshot

return;
}
DelayedMessageIndexBucketSnapshotFormat.DelayedIndex
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* 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.List;
import javax.annotation.concurrent.NotThreadSafe;
import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.DelayedIndex;
import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat.SnapshotSegment;

@NotThreadSafe
class CombinedSegmentDelayedIndexQueue implements DelayedIndexQueue {

private final List<SnapshotSegment> segmentListA;
private final List<SnapshotSegment> segmentListB;

private int segmentListACursor = 0;
private int segmentListBCursor = 0;
private int segmentACursor = 0;
private int segmentBCursor = 0;

private CombinedSegmentDelayedIndexQueue(List<SnapshotSegment> segmentListA,
List<SnapshotSegment> segmentListB) {
this.segmentListA = segmentListA;
this.segmentListB = segmentListB;
}

public static CombinedSegmentDelayedIndexQueue wrap(
List<SnapshotSegment> segmentListA,
List<SnapshotSegment> segmentListB) {
return new CombinedSegmentDelayedIndexQueue(segmentListA, segmentListB);
}

@Override
public boolean isEmpty() {
return segmentListACursor >= segmentListA.size() && segmentListBCursor >= segmentListB.size();
}

@Override
public DelayedIndex peek() {
return getValue(false);
}

@Override
public DelayedIndex pop() {
return getValue(true);
}

private DelayedIndex getValue(boolean needAdvanceCursor) {
// skip empty segment
while (segmentListACursor < segmentListA.size()
&& segmentListA.get(segmentListACursor).getIndexesCount() == 0) {
segmentListACursor++;
}
while (segmentListBCursor < segmentListB.size()
&& segmentListB.get(segmentListBCursor).getIndexesCount() == 0) {
segmentListBCursor++;
}

DelayedIndex delayedIndexA = null;
DelayedIndex delayedIndexB = null;
if (segmentListACursor >= segmentListA.size()) {
delayedIndexB = segmentListB.get(segmentListBCursor).getIndexes(segmentBCursor);
} else if (segmentListBCursor >= segmentListB.size()) {
delayedIndexA = segmentListA.get(segmentListACursor).getIndexes(segmentACursor);
} else {
delayedIndexA = segmentListA.get(segmentListACursor).getIndexes(segmentACursor);
delayedIndexB = segmentListB.get(segmentListBCursor).getIndexes(segmentBCursor);
}

DelayedIndex resultValue;
if (delayedIndexB == null || (delayedIndexA != null && COMPARATOR.compare(delayedIndexA, delayedIndexB) < 0)) {
resultValue = delayedIndexA;
if (needAdvanceCursor) {
if (++segmentACursor >= segmentListA.get(segmentListACursor).getIndexesCount()) {
segmentListA.set(segmentListACursor, null);
++segmentListACursor;
segmentACursor = 0;
}
}
} else {
resultValue = delayedIndexB;
if (needAdvanceCursor) {
if (++segmentBCursor >= segmentListB.get(segmentListBCursor).getIndexesCount()) {
segmentListB.set(segmentListBCursor, null);
++segmentListBCursor;
segmentBCursor = 0;
}
}
}

return resultValue;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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.Comparator;
import java.util.Objects;
import org.apache.pulsar.broker.delayed.proto.DelayedMessageIndexBucketSnapshotFormat;

interface DelayedIndexQueue {
Comparator<DelayedMessageIndexBucketSnapshotFormat.DelayedIndex> COMPARATOR = (o1, o2) -> {
if (!Objects.equals(o1.getTimestamp(), o2.getTimestamp())) {
return Long.compare(o1.getTimestamp(), o2.getTimestamp());
} else if (!Objects.equals(o1.getLedgerId(), o2.getLedgerId())) {
return Long.compare(o1.getLedgerId(), o2.getLedgerId());
} else {
return Long.compare(o1.getEntryId(), o2.getEntryId());
}
};

boolean isEmpty();

DelayedMessageIndexBucketSnapshotFormat.DelayedIndex peek();

DelayedMessageIndexBucketSnapshotFormat.DelayedIndex pop();
}
Loading