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

[fix][broker] Handle BucketDelayedDeliveryTracker recover failed #22735

Merged
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -72,7 +72,8 @@ public void initialize(PulsarService pulsarService) throws Exception {
}

@Override
public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) {
public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher)
throws RecoverDelayedDeliveryTrackerException {
return new BucketDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict,
bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket,
TimeUnit.SECONDS.toMillis(delayedDeliveryMaxTimeStepPerBucketSnapshotSegmentSeconds),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@
@Beta
public interface DelayedDeliveryTracker extends AutoCloseable {

DelayedDeliveryTracker DISABLE = new DelayedDeliveryTrackerDisable();

/**
* Add a message to the tracker.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* 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.impl.PositionImpl;

import java.util.NavigableSet;
import java.util.concurrent.CompletableFuture;

class DelayedDeliveryTrackerDisable implements DelayedDeliveryTracker {
dao-jun marked this conversation as resolved.
Show resolved Hide resolved

@Override
public boolean addMessage(long ledgerId, long entryId, long deliveryAt) {
return false;
}

@Override
public boolean hasMessageAvailable() {
return false;
}

@Override
public long getNumberOfDelayedMessages() {
return 0;
}

@Override
public long getBufferMemoryUsage() {
return 0;
}

@Override
public NavigableSet<PositionImpl> getScheduledMessages(int maxMessages) {
return null;
}

@Override
public boolean shouldPauseAllDeliveries() {
return false;
}

@Override
public void resetTickTime(long tickTime) {

}

@Override
public CompletableFuture<Void> clear() {
return null;
}

@Override
public void close() {

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,10 @@ public interface DelayedDeliveryTrackerFactory extends AutoCloseable {
*
* @param dispatcher
* a multi-consumer dispatcher instance
* @throws RecoverDelayedDeliveryTrackerException if the tracker cannot be recovered
*/
DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher);
DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher)
throws RecoverDelayedDeliveryTrackerException;

/**
* Close the factory and release all the resources.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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;

public class RecoverDelayedDeliveryTrackerException extends Exception {
public RecoverDelayedDeliveryTrackerException(Throwable cause) {
super(cause);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@
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.RecoverDelayedDeliveryTrackerException;
import org.apache.pulsar.broker.delayed.proto.DelayedIndex;
import org.apache.pulsar.broker.delayed.proto.SnapshotSegment;
import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
Expand Down Expand Up @@ -105,22 +106,24 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker
private CompletableFuture<Void> pendingLoad = null;

public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
Timer timer, long tickTimeMillis,
boolean isDelayedDeliveryDeliverAtTimeStrict,
BucketSnapshotStorage bucketSnapshotStorage,
long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis,
int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) {
Timer timer, long tickTimeMillis,
boolean isDelayedDeliveryDeliverAtTimeStrict,
BucketSnapshotStorage bucketSnapshotStorage,
long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis,
int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets)
throws RecoverDelayedDeliveryTrackerException {
this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict,
bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegmentInMillis,
maxIndexesPerBucketSnapshotSegment, maxNumBuckets);
}

public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher,
Timer timer, long tickTimeMillis, Clock clock,
boolean isDelayedDeliveryDeliverAtTimeStrict,
BucketSnapshotStorage bucketSnapshotStorage,
long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis,
int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) {
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);
this.minIndexCountPerBucket = minIndexCountPerBucket;
this.timeStepPerBucketSnapshotSegmentInMillis = timeStepPerBucketSnapshotSegmentInMillis;
Expand All @@ -136,7 +139,7 @@ public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispat
this.numberDelayedMessages = recoverBucketSnapshot();
}

private synchronized long recoverBucketSnapshot() throws RuntimeException {
private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryTrackerException {
ManagedCursor cursor = this.lastMutableBucket.getCursor();
Map<String, String> cursorProperties = cursor.getCursorProperties();
if (MapUtils.isEmpty(cursorProperties)) {
Expand Down Expand Up @@ -181,7 +184,7 @@ private synchronized long recoverBucketSnapshot() throws RuntimeException {
if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw new RuntimeException(e);
throw new RecoverDelayedDeliveryTrackerException(e);
}

for (Map.Entry<Range<Long>, CompletableFuture<List<DelayedIndex>>> entry : futures.entrySet()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@
import org.apache.pulsar.broker.cache.BundlesQuotas;
import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory;
import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerLoader;
import org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTrackerFactory;
import org.apache.pulsar.broker.intercept.BrokerInterceptor;
import org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImpl;
import org.apache.pulsar.broker.loadbalance.LoadManager;
Expand Down Expand Up @@ -288,6 +289,8 @@ public class BrokerService implements Closeable {
@Getter
@VisibleForTesting
private final DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory;
// inMemoryRedeliveryTrackerFactory is for the purpose of fallback if recover BucketDelayedDeliveryTracker failed.
private volatile DelayedDeliveryTrackerFactory fallbackRedeliveryTrackerFactory;
private final ServerBootstrap defaultServerBootstrap;
private final List<EventLoopGroup> protocolHandlersWorkerGroups = new ArrayList<>();

Expand Down Expand Up @@ -838,6 +841,9 @@ public CompletableFuture<Void> closeAsync() {
pendingLookupOperationsCounter.close();
try {
delayedDeliveryTrackerFactory.close();
if (fallbackRedeliveryTrackerFactory != null) {
fallbackRedeliveryTrackerFactory.close();
}
} catch (Exception e) {
log.warn("Error in closing delayedDeliveryTrackerFactory", e);
}
Expand Down Expand Up @@ -3390,6 +3396,25 @@ public void unblockDispatchersOnUnAckMessages(List<PersistentDispatcherMultipleC
}
}

/**
* Initializes the in-memory delayed delivery tracker factory when
* BucketDelayedDeliveryTrackerFactory.newTracker failed.
*/
public synchronized void initializeFallbackDelayedDeliveryTrackerFactory() {
if (fallbackRedeliveryTrackerFactory != null) {
return;
}

DelayedDeliveryTrackerFactory factory = new InMemoryDelayedDeliveryTrackerFactory();
try {
factory.initialize(pulsar);
this.fallbackRedeliveryTrackerFactory = factory;
} catch (Exception e) {
// it should never go here
log.error("Failed to initialize InMemoryDelayedDeliveryTrackerFactory", e);
}
}

private static class ConfigField {
// field holds the pulsar dynamic configuration.
final Field field;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,9 +50,11 @@
import org.apache.pulsar.broker.delayed.DelayedDeliveryTracker;
import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory;
import org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker;
import org.apache.pulsar.broker.delayed.RecoverDelayedDeliveryTrackerException;
import org.apache.pulsar.broker.delayed.bucket.BucketDelayedDeliveryTracker;
import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.service.BrokerServiceException;
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException;
import org.apache.pulsar.broker.service.Consumer;
Expand Down Expand Up @@ -1156,15 +1158,14 @@ public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata
}

synchronized (this) {
if (!delayedDeliveryTracker.isPresent()) {
if (delayedDeliveryTracker.isEmpty()) {
if (!msgMetadata.hasDeliverAtTime()) {
// No need to initialize the tracker here
return false;
}

// Initialize the tracker the first time we need to use it
delayedDeliveryTracker = Optional
.of(topic.getBrokerService().getDelayedDeliveryTrackerFactory().newTracker(this));
delayedDeliveryTracker = Optional.of(initializeDelayedDeliveryTracer());
}

delayedDeliveryTracker.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis());
Expand All @@ -1174,6 +1175,47 @@ public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata
}
}

/**
* Initialize the delayed delivery tracker.
*
* @return
*/
private DelayedDeliveryTracker initializeDelayedDeliveryTracer() {
DelayedDeliveryTracker tracker = DelayedDeliveryTracker.DISABLE;

BrokerService brokerService = topic.getBrokerService();
DelayedDeliveryTrackerFactory factory = brokerService.getDelayedDeliveryTrackerFactory();

if (factory instanceof BucketDelayedDeliveryTrackerFactory) {
try {
tracker = factory.newTracker(this);
} catch (Exception e) {
dao-jun marked this conversation as resolved.
Show resolved Hide resolved
// If failed to create BucketDelayedDeliveryTracker, fallback to InMemoryDelayedDeliveryTracker
log.warn("Failed to create BucketDelayedDeliveryTracker", e);
dao-jun marked this conversation as resolved.
Show resolved Hide resolved
try {
brokerService.initializeFallbackDelayedDeliveryTrackerFactory();
tracker = brokerService.getFallbackRedeliveryTrackerFactory().newTracker(this);
} catch (RecoverDelayedDeliveryTrackerException ex) {
dao-jun marked this conversation as resolved.
Show resolved Hide resolved
// it should never goes here
log.warn("Failed to fallback to InMemoryDelayedDeliveryTracker, topic: {}, subscription: {}",
topic.getName(), subscription.getName(), ex);
}
}

return tracker;
}

try {
tracker = factory.newTracker(this);
} catch (RecoverDelayedDeliveryTrackerException ex) {
// it should never goes here
log.warn("Failed to create InMemoryDelayedDeliveryTracker, topic: {}, subscription: {}",
topic.getName(), subscription.getName(), ex);
}

return tracker;
}

protected synchronized NavigableSet<PositionImpl> getMessagesToReplayNow(int maxMessagesToRead) {
if (delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().hasMessageAvailable()) {
delayedDeliveryTracker.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,7 @@ public void testContainsMessage(BucketDelayedDeliveryTracker tracker) {
}

@Test(dataProvider = "delayedTracker", invocationCount = 10)
public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) {
public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) throws Exception {
for (int i = 1; i <= 100; i++) {
tracker.addMessage(i, i, i * 10);
}
Expand Down Expand Up @@ -265,7 +265,7 @@ public void testRoaringBitmapSerialize() {
}

@Test(dataProvider = "delayedTracker")
public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) {
public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) throws Exception {
for (int i = 1; i <= 110; i++) {
tracker.addMessage(i, i, i * 10);
Awaitility.await().untilAsserted(() -> {
Expand Down Expand Up @@ -318,7 +318,7 @@ public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) {
}

@Test(dataProvider = "delayedTracker")
public void testWithBkException(final BucketDelayedDeliveryTracker tracker) {
public void testWithBkException(final BucketDelayedDeliveryTracker tracker) throws Exception {
MockBucketSnapshotStorage mockBucketSnapshotStorage = (MockBucketSnapshotStorage) bucketSnapshotStorage;
mockBucketSnapshotStorage.injectCreateException(
new BucketSnapshotPersistenceException("Bookie operation timeout, op: Create entry"));
Expand Down
Loading