Skip to content

[fix] [broker] Fix incorrect stat sub.msgBacklogNoDelayed if enabled delayedDeliveryFixedDelayDetectionLookahead #23353

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

Open
wants to merge 5 commits into
base: master
Choose a base branch
from
Open
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 @@ -151,6 +151,10 @@ default boolean checkAndUnblockIfStuck() {
*/
default void afterAckMessages(Throwable exOfDeletion, Object ctxOfDeletion){}

default boolean isAllWaitingReadMessagesAreFixedDelayed() {
return false;
}

/**
* Trigger a new "readMoreEntries" if the dispatching has been paused before. This method is only implemented in
* {@link org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers} right now, other
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1334,6 +1334,11 @@ protected synchronized boolean shouldPauseDeliveryForDelayTracker() {
return delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().shouldPauseAllDeliveries();
}

@Override
public boolean isAllWaitingReadMessagesAreFixedDelayed() {
return shouldPauseDeliveryForDelayTracker();
}
Copy link
Contributor

Choose a reason for hiding this comment

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

pendingReadMessageAreFixedDelayed?


@Override
public long getNumberOfDelayedMessages() {
return delayedDeliveryTracker.map(DelayedDeliveryTracker::getNumberOfDelayedMessages).orElse(0L);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1269,16 +1269,27 @@ public CompletableFuture<SubscriptionStatsImpl> getStatsAsync(GetStatsOptions ge
((PersistentDispatcherMultipleConsumers) dispatcher).getBucketDelayedIndexStats();
}

subStats.msgBacklog = getNumberOfEntriesInBacklog(getStatsOptions.isGetPreciseBacklog());
if (Subscription.isIndividualAckMode(subType)) {
if (dispatcher instanceof PersistentDispatcherMultipleConsumers) {
PersistentDispatcherMultipleConsumers d = (PersistentDispatcherMultipleConsumers) dispatcher;
subStats.unackedMessages = d.getTotalUnackedMessages();
subStats.blockedSubscriptionOnUnackedMsgs = d.isBlockedDispatcherOnUnackedMsgs();
subStats.msgDelayed = d.getNumberOfDelayedMessages();
subStats.msgInReplay = d.getNumberOfMessagesInReplay();
if (d.isAllWaitingReadMessagesAreFixedDelayed()) {
long msgDeliveredOut = 0;
for (Consumer c : dispatcher.getConsumers()){
msgDeliveredOut += c.getUnackedMessages();
}
subStats.msgDelayed = subStats.msgBacklog - msgDeliveredOut - subStats.msgInReplay;
subStats.msgDelayedInMemory = d.getNumberOfDelayedMessages();
} else {
subStats.msgDelayed = d.getNumberOfDelayedMessages();
subStats.msgDelayedInMemory = subStats.msgDelayed;
}
}
}
subStats.msgBacklog = getNumberOfEntriesInBacklog(getStatsOptions.isGetPreciseBacklog());

if (getStatsOptions.isSubscriptionBacklogSize()) {
subStats.backlogSize = topic.getManagedLedger()
.getEstimatedBacklogSize(cursor.getMarkDeletedPosition());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
import io.opentelemetry.api.common.Attributes;
import java.time.Duration;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
Expand All @@ -36,21 +37,26 @@
import lombok.Cleanup;
import org.apache.bookkeeper.client.BKException;
import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory;
import org.apache.pulsar.broker.service.Dispatcher;
import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil;
import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats;
import org.apache.pulsar.broker.testcontext.PulsarTestContext;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerConsumerBase;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies;
import org.apache.pulsar.common.policies.data.SubscriptionStats;
import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue;
import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes;
import org.awaitility.Awaitility;
import org.awaitility.reflect.WhiteboxImpl;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
Expand Down Expand Up @@ -684,4 +690,92 @@ public void testDelayedDeliveryExceedsMaxDelay() throws Exception {
+ maxDeliveryDelayInMillis + " milliseconds");
}
}

@Test
public void testMsgBacklogNoDelayedWhenFixedDelay() throws Exception {
final long originalDelayedDeliveryFixedDelayDetectionLookahead =
pulsar.getConfig().getDelayedDeliveryFixedDelayDetectionLookahead();;
final int delayedDeliveryFixedDelayDetectionLookahead = 10;
final int msgNoDelayed = 15;
final int receiverQueueSize = 10;
final int msgDelayed = delayedDeliveryFixedDelayDetectionLookahead * 10;
final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp");
final String sName = "delayed_s1";
DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory =
WhiteboxImpl.getInternalState(pulsar.getBrokerService(), "delayedDeliveryTrackerFactory");
if (delayedDeliveryTrackerFactory != null) {
pulsar.getConfig()
.setDelayedDeliveryFixedDelayDetectionLookahead(delayedDeliveryFixedDelayDetectionLookahead);
delayedDeliveryTrackerFactory.initialize(pulsar);
}
admin.topics().createNonPartitionedTopic(topic);
admin.topics().createSubscription(topic, sName, MessageId.earliest);
PersistentTopic persistentTopic =
(PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get();

// Send some message no-delayed.
// Send many messages delayed.
Producer<String> producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create();
for (int i = 0; i < msgNoDelayed; i++) {
producer.newMessage().send();
}
for (int i = 0; i < msgDelayed; i++) {
producer.newMessage().deliverAfter(1, TimeUnit.HOURS).value(i + "").send();
}
Consumer<String> consumer1 = pulsarClient.newConsumer(Schema.STRING).topic(topic).subscriptionName(sName)
.receiverQueueSize(receiverQueueSize)
.subscriptionType(SubscriptionType.Shared).subscribe();
Consumer<String> consumer2 = pulsarClient.newConsumer(Schema.STRING).topic(topic).subscriptionName(sName)
.receiverQueueSize(receiverQueueSize)
.subscriptionType(SubscriptionType.Shared).subscribe();

// Wait for the checker that named "shouldPauseDeliveryForDelayTracker".
PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) persistentTopic
.getSubscription(sName).getDispatcher();
Awaitility.await().untilAsserted(() -> {
assertTrue(dispatcher.shouldPauseDeliveryForDelayTracker());
assertTrue(dispatcher.getNumberOfDelayedMessages() >= delayedDeliveryFixedDelayDetectionLookahead);
});
// Verify: backlog stats
SubscriptionStats subscriptionStats1 = persistentTopic.getStats(true, false, false)
.getSubscriptions().get(sName);
assertEquals(subscriptionStats1.getMsgBacklog(), msgDelayed + msgNoDelayed);
assertEquals(subscriptionStats1.getMsgDelayed(), msgDelayed);
assertEquals(subscriptionStats1.getMsgBacklogNoDelayed(), msgNoDelayed);
assertEquals(subscriptionStats1.getMsgInReplay(), 0);
assertTrue(subscriptionStats1.getMsgDelayedInMemory() < msgDelayed);

// Let some messages being pushed into replay queue.
consumer2.close();
// Wait for the checker that named "shouldPauseDeliveryForDelayTracker".
Awaitility.await().untilAsserted(() -> {
assertTrue(dispatcher.shouldPauseDeliveryForDelayTracker());
assertTrue(dispatcher.getNumberOfDelayedMessages() >= delayedDeliveryFixedDelayDetectionLookahead);
});
// And verify: backlog stats
Awaitility.await().atMost(Duration.ofSeconds(3600)).untilAsserted(() -> {
SubscriptionStats subscriptionStats2 = persistentTopic.getStats(true, false, false)
.getSubscriptions().get(sName);
assertEquals(subscriptionStats2.getMsgBacklog(), msgDelayed + msgNoDelayed);
assertEquals(subscriptionStats2.getMsgDelayed(), msgDelayed);
assertEquals(subscriptionStats2.getMsgBacklogNoDelayed(), msgNoDelayed);
GrowableArrayBlockingQueue incomingMessages =
WhiteboxImpl.getInternalState(consumer1, "incomingMessages");
assertEquals(subscriptionStats2.getMsgInReplay(), msgNoDelayed - incomingMessages.size());
assertTrue(subscriptionStats2.getMsgDelayedInMemory() < msgDelayed);
});

// cleanup.
DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory2 =
WhiteboxImpl.getInternalState(pulsar.getBrokerService(), "delayedDeliveryTrackerFactory");
if (delayedDeliveryTrackerFactory2 != null) {
pulsar.getConfig().setDelayedDeliveryFixedDelayDetectionLookahead(
originalDelayedDeliveryFixedDelayDetectionLookahead);
delayedDeliveryTrackerFactory2.initialize(pulsar);
}
consumer1.close();
consumer2.close();
producer.close();
admin.topics().delete(topic, false);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -63,9 +63,12 @@ public interface SubscriptionStats {
/** Flag to verify if subscription is blocked due to reaching threshold of unacked messages. */
boolean isBlockedSubscriptionOnUnackedMsgs();

/** Number of delayed messages currently being tracked. */
/** Number of delayed messages. */
long getMsgDelayed();

/** Number of delayed messages currently being tracked. */
long getMsgDelayedInMemory();

/** Number of messages registered for replay. */
long getMsgInReplay();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,9 +71,12 @@ public class SubscriptionStatsImpl implements SubscriptionStats {
/** Flag to verify if subscription is blocked due to reaching threshold of unacked messages. */
public boolean blockedSubscriptionOnUnackedMsgs;

/** Number of delayed messages currently being tracked. */
/** Number of delayed messages. */
public long msgDelayed;

/** Number of delayed messages currently being tracked. */
public long msgDelayedInMemory;

/** Number of messages registered for replay. */
public long msgInReplay;

Expand Down
Loading