diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 1b021bd569969..19e9ff625cada 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -817,6 +817,22 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + "The higher the number, the more equal the assignment of keys to consumers") private int subscriptionKeySharedConsistentHashingReplicaPoints = 100; + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For persistent Key_Shared subscriptions, enables the use of the classic implementation of the " + + "Key_Shared subscription that was used before Pulsar 4.0.0 and PIP-379.", + dynamic = true + ) + private boolean subscriptionKeySharedUseClassicPersistentImplementation = false; + + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For persistent Shared subscriptions, enables the use of the classic implementation of the Shared " + + "subscription that was used before Pulsar 4.0.0.", + dynamic = true + ) + private boolean subscriptionSharedUseClassicPersistentImplementation = false; + @FieldContext( category = CATEGORY_POLICIES, doc = "Set the default behavior for message deduplication in the broker.\n\n" 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 f93a627bca7b8..bec5134c4f79a 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 @@ -24,12 +24,12 @@ import java.time.Clock; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; @Slf4j public abstract class AbstractDelayedDeliveryTracker implements DelayedDeliveryTracker, TimerTask { - protected final PersistentDispatcherMultipleConsumers dispatcher; + protected final AbstractPersistentDispatcherMultipleConsumers dispatcher; // Reference to the shared (per-broker) timer for delayed delivery protected final Timer timer; @@ -49,13 +49,13 @@ public abstract class AbstractDelayedDeliveryTracker implements DelayedDeliveryT private final boolean isDelayedDeliveryDeliverAtTimeStrict; - public AbstractDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, + public AbstractDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict) { this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict); } - public AbstractDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, + public AbstractDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, Clock clock, boolean isDelayedDeliveryDeliverAtTimeStrict) { this.dispatcher = dispatcher; 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 11ad243e0c9d1..c2d002ad19cb0 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 @@ -36,7 +36,7 @@ import org.apache.pulsar.broker.delayed.bucket.BucketSnapshotStorage; import org.apache.pulsar.broker.delayed.bucket.RecoverDelayedDeliveryTrackerException; import org.apache.pulsar.broker.service.BrokerService; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,7 +78,7 @@ public void initialize(PulsarService pulsarService) throws Exception { } @Override - public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { + public DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher) { String topicName = dispatcher.getTopic().getName(); String subscriptionName = dispatcher.getSubscription().getName(); BrokerService brokerService = dispatcher.getTopic().getBrokerService(); @@ -97,7 +97,7 @@ public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers d } @VisibleForTesting - BucketDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers dispatcher) + BucketDelayedDeliveryTracker newTracker0(AbstractPersistentDispatcherMultipleConsumers dispatcher) throws RecoverDelayedDeliveryTrackerException { return new BucketDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java index 5427a46a2e4b3..763b6d66da142 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java @@ -20,7 +20,7 @@ import com.google.common.annotations.Beta; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; /** * Factory of InMemoryDelayedDeliveryTracker objects. This is the entry point for implementations. @@ -42,7 +42,7 @@ public interface DelayedDeliveryTrackerFactory extends AutoCloseable { * @param dispatcher * a multi-consumer dispatcher instance */ - DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher); + DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher); /** * Close the factory and release all the resources. 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 8bd9fafa13715..bdc6e4c814e33 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 @@ -28,7 +28,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; @Slf4j @@ -52,17 +52,18 @@ public class InMemoryDelayedDeliveryTracker extends AbstractDelayedDeliveryTrack // Track whether we have seen all messages with fixed delay so far. private boolean messagesHaveFixedDelay = true; - InMemoryDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, + InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, + long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, long fixedDelayDetectionLookahead) { this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); } - public InMemoryDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, - long tickTimeMillis, Clock clock, - boolean isDelayedDeliveryDeliverAtTimeStrict, - long fixedDelayDetectionLookahead) { + public InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, + long tickTimeMillis, Clock clock, + boolean isDelayedDeliveryDeliverAtTimeStrict, + long fixedDelayDetectionLookahead) { super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); this.fixedDelayDetectionLookahead = fixedDelayDetectionLookahead; } 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 179cf74db4179..f8b8f5a8ba459 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 @@ -25,7 +25,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,7 +51,7 @@ public void initialize(PulsarService pulsarService) { } @Override - public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { + public DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher) { String topicName = dispatcher.getTopic().getName(); String subscriptionName = dispatcher.getSubscription().getName(); DelayedDeliveryTracker tracker = DelayedDeliveryTracker.DISABLE; @@ -66,7 +66,7 @@ public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers d } @VisibleForTesting - InMemoryDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers dispatcher) { + InMemoryDelayedDeliveryTracker newTracker0(AbstractPersistentDispatcherMultipleConsumers dispatcher) { return new InMemoryDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); } 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 47c78fa9ee2ec..0091bf5b9bd30 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 @@ -57,7 +57,7 @@ import org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.proto.DelayedIndex; import org.apache.pulsar.broker.delayed.proto.SnapshotSegment; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; @@ -105,7 +105,7 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker private CompletableFuture pendingLoad = null; - public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, + public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, BucketSnapshotStorage bucketSnapshotStorage, @@ -117,7 +117,7 @@ public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispat maxIndexesPerBucketSnapshotSegment, maxNumBuckets); } - public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, + public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, Clock clock, boolean isDelayedDeliveryDeliverAtTimeStrict, BucketSnapshotStorage bucketSnapshotStorage, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java index 9fc6b9581a3ac..e3c2cf40cf318 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java @@ -24,10 +24,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * @@ -239,7 +236,4 @@ private int getFirstConsumerIndexOfPriority(int targetPriority) { return -1; } - private static final Logger log = LoggerFactory.getLogger(PersistentStickyKeyDispatcherMultipleConsumers.class); - - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 8d0b9a4a84e6a..fee5e25647ce6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -129,8 +129,8 @@ import org.apache.pulsar.broker.service.TopicEventsListener.EventStage; import org.apache.pulsar.broker.service.TopicEventsListener.TopicEvent; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.SystemTopic; import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; @@ -301,7 +301,7 @@ public class BrokerService implements Closeable { private final int maxUnackedMessages; public final int maxUnackedMsgsPerDispatcher; private final AtomicBoolean blockedDispatcherOnHighUnackedMsgs = new AtomicBoolean(false); - private final Set blockedDispatchers = ConcurrentHashMap.newKeySet(); + private final Set blockedDispatchers = ConcurrentHashMap.newKeySet(); private final ReadWriteLock lock = new ReentrantReadWriteLock(); @VisibleForTesting private final DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory; @@ -3328,7 +3328,7 @@ public OrderedExecutor getTopicOrderedExecutor() { * @param dispatcher * @param numberOfMessages */ - public void addUnAckedMessages(PersistentDispatcherMultipleConsumers dispatcher, int numberOfMessages) { + public void addUnAckedMessages(AbstractPersistentDispatcherMultipleConsumers dispatcher, int numberOfMessages) { // don't block dispatchers if maxUnackedMessages = 0 if (maxUnackedMessages > 0) { totalUnackedMessages.add(numberOfMessages); @@ -3387,10 +3387,10 @@ private void blockDispatchersWithLargeUnAckMessages() { try { forEachTopic(topic -> { topic.getSubscriptions().forEach((subName, persistentSubscription) -> { - if (persistentSubscription.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentSubscription - .getDispatcher(); + if (persistentSubscription.getDispatcher() + instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentSubscription.getDispatcher(); int dispatcherUnAckMsgs = dispatcher.getTotalUnackedMessages(); if (dispatcherUnAckMsgs > maxUnackedMsgsPerDispatcher) { log.info("[{}] Blocking dispatcher due to reached max broker limit {}", @@ -3411,7 +3411,7 @@ private void blockDispatchersWithLargeUnAckMessages() { * * @param dispatcherList */ - public void unblockDispatchersOnUnAckMessages(List dispatcherList) { + public void unblockDispatchersOnUnAckMessages(List dispatcherList) { lock.writeLock().lock(); try { dispatcherList.forEach(dispatcher -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index c9584f2c1790f..d25ebd0839df1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -146,7 +146,7 @@ public class Consumer { private static final double avgPercent = 0.9; private boolean preciseDispatcherFlowControl; - private Position lastSentPositionWhenJoining; + private Position readPositionWhenJoining; private final String clientAddress; // IP address only, no port number included private final MessageId startMessageId; private final boolean isAcknowledgmentAtBatchIndexLevelEnabled; @@ -973,8 +973,8 @@ public ConsumerStatsImpl getStats() { stats.unackedMessages = unackedMessages; stats.blockedConsumerOnUnackedMsgs = blockedConsumerOnUnackedMsgs; stats.avgMessagesPerEntry = getAvgMessagesPerEntry(); - if (lastSentPositionWhenJoining != null) { - stats.lastSentPositionWhenJoining = lastSentPositionWhenJoining.toString(); + if (readPositionWhenJoining != null) { + stats.readPositionWhenJoining = readPositionWhenJoining.toString(); } return stats; } @@ -1189,8 +1189,8 @@ public boolean isPreciseDispatcherFlowControl() { return preciseDispatcherFlowControl; } - public void setLastSentPositionWhenJoining(Position lastSentPositionWhenJoining) { - this.lastSentPositionWhenJoining = lastSentPositionWhenJoining; + public void setReadPositionWhenJoining(Position readPositionWhenJoining) { + this.readPositionWhenJoining = readPositionWhenJoining; } public int getMaxUnackedMessages() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java index d1d44709a9c52..f68a9a0986b84 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java @@ -153,8 +153,8 @@ default void afterAckMessages(Throwable exOfDeletion, Object ctxOfDeletion){} /** * 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 - * implements are not necessary to implement this method. + * {@link org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers} right now, + * other implementations do not necessary implement this method. * @return did a resume. */ default boolean checkAndResumeIfPaused(){ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java new file mode 100644 index 0000000000000..79ce308158422 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.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.service; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.bookkeeper.mledger.Position; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.api.proto.KeySharedMode; + +public interface StickyKeyDispatcher extends Dispatcher { + + boolean hasSameKeySharedPolicy(KeySharedMeta ksm); + + Map> getConsumerKeyHashRanges(); + + boolean isAllowOutOfOrderDelivery(); + + KeySharedMode getKeySharedMode(); + + StickyKeyConsumerSelector getSelector(); + + long getNumberOfMessagesInReplay(); + + default LinkedHashMap getRecentlyJoinedConsumers() { + return null; + } + + boolean isClassic(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java new file mode 100644 index 0000000000000..79d365b9fee21 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java @@ -0,0 +1,67 @@ +/* + * 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.service.persistent; + +import java.util.Map; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; + +public abstract class AbstractPersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers + implements Dispatcher, AsyncCallbacks.ReadEntriesCallback { + public AbstractPersistentDispatcherMultipleConsumers(Subscription subscription, + ServiceConfiguration serviceConfig) { + super(subscription, serviceConfig); + } + + public abstract void unBlockDispatcherOnUnackedMsgs(); + + public abstract void readMoreEntriesAsync(); + + public abstract String getName(); + + public abstract boolean isBlockedDispatcherOnUnackedMsgs(); + + public abstract int getTotalUnackedMessages(); + + public abstract void blockDispatcherOnUnackedMsgs(); + + public abstract long getNumberOfMessagesInReplay(); + + public abstract boolean isHavePendingRead(); + + public abstract boolean isHavePendingReplayRead(); + + public abstract ManagedCursor getCursor(); + + public abstract Topic getTopic(); + + public abstract Subscription getSubscription(); + + public abstract long getDelayedTrackerMemoryUsage(); + + public abstract Map getBucketDelayedIndexStats(); + + public abstract boolean isClassic(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index b34a0b454385f..46f1f0a535650 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -41,12 +41,18 @@ public class MessageRedeliveryController { private final boolean allowOutOfOrderDelivery; + private final boolean isClassicDispatcher; private final ConcurrentBitmapSortedLongPairSet messagesToRedeliver; private final ConcurrentLongLongPairHashMap hashesToBeBlocked; private final ConcurrentLongLongHashMap hashesRefCount; public MessageRedeliveryController(boolean allowOutOfOrderDelivery) { + this(allowOutOfOrderDelivery, false); + } + + public MessageRedeliveryController(boolean allowOutOfOrderDelivery, boolean isClassicDispatcher) { this.allowOutOfOrderDelivery = allowOutOfOrderDelivery; + this.isClassicDispatcher = isClassicDispatcher; this.messagesToRedeliver = new ConcurrentBitmapSortedLongPairSet(); if (!allowOutOfOrderDelivery) { this.hashesToBeBlocked = ConcurrentLongLongPairHashMap @@ -65,7 +71,7 @@ public void add(long ledgerId, long entryId) { public void add(long ledgerId, long entryId, long stickyKeyHash) { if (!allowOutOfOrderDelivery) { - if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + if (!isClassicDispatcher && stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { throw new IllegalArgumentException("Sticky key hash is not set. It is required."); } boolean inserted = hashesToBeBlocked.putIfAbsent(ledgerId, entryId, stickyKeyHash, 0); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 73d152bab1a60..b1cd186c31784 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -39,7 +39,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.function.Predicate; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; @@ -60,7 +59,6 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.Consumer; -import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryAndMetadata; import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; @@ -85,8 +83,7 @@ /** * */ -public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers - implements Dispatcher, ReadEntriesCallback { +public class PersistentDispatcherMultipleConsumers extends AbstractPersistentDispatcherMultipleConsumers { protected final PersistentTopic topic; protected final ManagedCursor cursor; protected volatile Range lastIndividualDeletedRangeFromCursorRecovery; @@ -162,7 +159,7 @@ public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCurso this.name = topic.getName() + " / " + Codec.decode(cursor.getName()); this.topic = topic; this.dispatchMessagesThread = topic.getBrokerService().getTopicOrderedExecutor().chooseThread(); - this.redeliveryMessages = new MessageRedeliveryController(allowOutOfOrderDelivery); + this.redeliveryMessages = new MessageRedeliveryController(allowOutOfOrderDelivery, false); this.redeliveryTracker = this.serviceConfig.isSubscriptionRedeliveryTrackerEnabled() ? new InMemoryRedeliveryTracker() : RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; @@ -320,6 +317,7 @@ private synchronized void internalConsumerFlow(Consumer consumer, int additional * We should not call readMoreEntries() recursively in the same thread as there is a risk of StackOverflowError. * */ + @Override public void readMoreEntriesAsync() { // deduplication for readMoreEntriesAsync calls if (readMoreEntriesAsyncRequested.compareAndSet(false, true)) { @@ -1285,6 +1283,7 @@ public void blockDispatcherOnUnackedMsgs() { blockedDispatcherOnUnackedMsgs = TRUE; } + @Override public void unBlockDispatcherOnUnackedMsgs() { blockedDispatcherOnUnackedMsgs = FALSE; } @@ -1293,6 +1292,7 @@ public int getTotalUnackedMessages() { return totalUnackedMessages; } + @Override public String getName() { return name; } @@ -1488,6 +1488,11 @@ public Map getBucketDelayedIndexStats() { return Collections.emptyMap(); } + @Override + public boolean isClassic() { + return false; + } + public ManagedCursor getCursor() { return cursor; } @@ -1505,5 +1510,15 @@ public long getNumberOfMessagesInReplay() { return redeliveryMessages.size(); } + @Override + public boolean isHavePendingRead() { + return havePendingRead; + } + + @Override + public boolean isHavePendingReplayRead() { + return havePendingReplayRead; + } + private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumers.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java new file mode 100644 index 0000000000000..6ab7acfa56da8 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java @@ -0,0 +1,1374 @@ +/* + * 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.service.persistent; + +import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.function.Predicate; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException; +import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; +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.bucket.BucketDelayedDeliveryTracker; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; +import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryAndMetadata; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; +import org.apache.pulsar.broker.service.EntryBatchSizes; +import org.apache.pulsar.broker.service.InMemoryRedeliveryTracker; +import org.apache.pulsar.broker.service.RedeliveryTracker; +import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; +import org.apache.pulsar.broker.service.SendMessageInfo; +import org.apache.pulsar.broker.service.SharedConsumerAssignor; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; +import org.apache.pulsar.broker.transaction.exception.buffer.TransactionBufferException; +import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.Codec; +import org.apache.pulsar.common.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the "classic" dispatcher implementation for the Shared subscription that was used before + * Pulsar 4.0.0 and PIP-379. When `subscriptionSharedUseClassicPersistentImplementation=true`, + * this dispatcher will be used. The main purpose is to provide a way to rollback to the old behavior + * in case of issues with the preferred implementation. + */ +public class PersistentDispatcherMultipleConsumersClassic extends AbstractPersistentDispatcherMultipleConsumers { + protected final PersistentTopic topic; + protected final ManagedCursor cursor; + protected volatile Range lastIndividualDeletedRangeFromCursorRecovery; + + private CompletableFuture closeFuture = null; + protected final MessageRedeliveryController redeliveryMessages; + protected final RedeliveryTracker redeliveryTracker; + + private Optional delayedDeliveryTracker = Optional.empty(); + + protected volatile boolean havePendingRead = false; + protected volatile boolean havePendingReplayRead = false; + protected volatile Position minReplayedPosition = null; + protected boolean shouldRewindBeforeReadingOrReplaying = false; + protected final String name; + private boolean sendInProgress = false; + protected static final AtomicIntegerFieldUpdater + TOTAL_AVAILABLE_PERMITS_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "totalAvailablePermits"); + protected volatile int totalAvailablePermits = 0; + protected volatile int readBatchSize; + protected final Backoff readFailureBackoff; + private static final AtomicIntegerFieldUpdater + TOTAL_UNACKED_MESSAGES_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "totalUnackedMessages"); + protected volatile int totalUnackedMessages = 0; + /** + * A signature that relate to the check of "Dispatching has paused on cursor data can fully persist". + * Note: It is a tool that helps determine whether it should trigger a new reading after acknowledgments to avoid + * too many CPU circles, see {@link #afterAckMessages(Throwable, Object)} for more details. Do not use this + * to confirm whether the delivery should be paused, please call {@link #shouldPauseOnAckStatePersist}. + */ + protected static final AtomicIntegerFieldUpdater + BLOCKED_DISPATCHER_ON_CURSOR_DATA_CAN_NOT_FULLY_PERSIST_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "blockedDispatcherOnCursorDataCanNotFullyPersist"); + private volatile int blockedDispatcherOnCursorDataCanNotFullyPersist = FALSE; + private volatile int blockedDispatcherOnUnackedMsgs = FALSE; + protected static final AtomicIntegerFieldUpdater + BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "blockedDispatcherOnUnackedMsgs"); + protected Optional dispatchRateLimiter = Optional.empty(); + private AtomicBoolean isRescheduleReadInProgress = new AtomicBoolean(false); + protected final ExecutorService dispatchMessagesThread; + private final SharedConsumerAssignor assignor; + + protected enum ReadType { + Normal, Replay + } + + public PersistentDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, + Subscription subscription) { + this(topic, cursor, subscription, true); + } + + public PersistentDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, + Subscription subscription, + boolean allowOutOfOrderDelivery) { + super(subscription, topic.getBrokerService().pulsar().getConfiguration()); + this.cursor = cursor; + this.lastIndividualDeletedRangeFromCursorRecovery = cursor.getLastIndividualDeletedRange(); + this.name = topic.getName() + " / " + Codec.decode(cursor.getName()); + this.topic = topic; + this.dispatchMessagesThread = topic.getBrokerService().getTopicOrderedExecutor().chooseThread(); + this.redeliveryMessages = new MessageRedeliveryController(allowOutOfOrderDelivery, true); + this.redeliveryTracker = this.serviceConfig.isSubscriptionRedeliveryTrackerEnabled() + ? new InMemoryRedeliveryTracker() + : RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; + this.readBatchSize = serviceConfig.getDispatcherMaxReadBatchSize(); + this.initializeDispatchRateLimiterIfNeeded(); + this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addMessageToReplay); + this.readFailureBackoff = new Backoff( + topic.getBrokerService().pulsar().getConfiguration().getDispatcherReadFailureBackoffInitialTimeInMs(), + TimeUnit.MILLISECONDS, + 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + } + + @Override + public synchronized CompletableFuture addConsumer(Consumer consumer) { + if (IS_CLOSED_UPDATER.get(this) == TRUE) { + log.warn("[{}] Dispatcher is already closed. Closing consumer {}", name, consumer); + consumer.disconnect(); + return CompletableFuture.completedFuture(null); + } + if (consumerList.isEmpty()) { + if (havePendingRead || havePendingReplayRead) { + // There is a pending read from previous run. We must wait for it to complete and then rewind + shouldRewindBeforeReadingOrReplaying = true; + } else { + cursor.rewind(); + shouldRewindBeforeReadingOrReplaying = false; + } + redeliveryMessages.clear(); + delayedDeliveryTracker.ifPresent(tracker -> { + // Don't clean up BucketDelayedDeliveryTracker, otherwise we will lose the bucket snapshot + if (tracker instanceof InMemoryDelayedDeliveryTracker) { + tracker.clear(); + } + }); + } + + if (isConsumersExceededOnSubscription()) { + log.warn("[{}] Attempting to add consumer to subscription which reached max consumers limit {}", + name, consumer); + return FutureUtil.failedFuture(new ConsumerBusyException("Subscription reached max consumers limit")); + } + // This is not an expected scenario, it will never happen in expected. Just print a warn log if the unexpected + // scenario happens. See more detail: https://github.com/apache/pulsar/pull/22283. + if (consumerSet.contains(consumer)) { + log.warn("[{}] Attempting to add a consumer that already registered {}", name, consumer); + } + + consumerList.add(consumer); + if (consumerList.size() > 1 + && consumer.getPriorityLevel() < consumerList.get(consumerList.size() - 2).getPriorityLevel()) { + consumerList.sort(Comparator.comparingInt(Consumer::getPriorityLevel)); + } + consumerSet.add(consumer); + + return CompletableFuture.completedFuture(null); + } + + @Override + protected boolean isConsumersExceededOnSubscription() { + return isConsumersExceededOnSubscription(topic, consumerList.size()); + } + + @Override + public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { + // decrement unack-message count for removed consumer + addUnAckedMessages(-consumer.getUnackedMessages()); + if (consumerSet.removeAll(consumer) == 1) { + consumerList.remove(consumer); + log.info("Removed consumer {} with pending {} acks", consumer, consumer.getPendingAcks().size()); + if (consumerList.isEmpty()) { + clearComponentsAfterRemovedAllConsumers(); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Consumer are left, reading more entries", name); + } + consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + }); + totalAvailablePermits -= consumer.getAvailablePermits(); + if (log.isDebugEnabled()) { + log.debug("[{}] Decreased totalAvailablePermits by {} in PersistentDispatcherMultipleConsumers. " + + "New dispatcher permit count is {}", name, consumer.getAvailablePermits(), + totalAvailablePermits); + } + readMoreEntries(); + } + } else { + /** + * This is not an expected scenario, it will never happen in expected. + * Just add a defensive code to avoid the topic can not be unloaded anymore: remove the consumers which + * are not mismatch with {@link #consumerSet}. See more detail: https://github.com/apache/pulsar/pull/22270. + */ + log.error("[{}] Trying to remove a non-connected consumer: {}", name, consumer); + consumerList.removeIf(c -> consumer.equals(c)); + if (consumerList.isEmpty()) { + clearComponentsAfterRemovedAllConsumers(); + } + } + } + + private synchronized void clearComponentsAfterRemovedAllConsumers() { + cancelPendingRead(); + + redeliveryMessages.clear(); + redeliveryTracker.clear(); + if (closeFuture != null) { + log.info("[{}] All consumers removed. Subscription is disconnected", name); + closeFuture.complete(null); + } + totalAvailablePermits = 0; + } + + @Override + public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) { + topic.getBrokerService().executor().execute(() -> { + internalConsumerFlow(consumer, additionalNumberOfMessages); + }); + } + + private synchronized void internalConsumerFlow(Consumer consumer, int additionalNumberOfMessages) { + if (!consumerSet.contains(consumer)) { + if (log.isDebugEnabled()) { + log.debug("[{}] Ignoring flow control from disconnected consumer {}", name, consumer); + } + return; + } + + totalAvailablePermits += additionalNumberOfMessages; + + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Trigger new read after receiving flow control message with permits {} " + + "after adding {} permits", name, consumer, + totalAvailablePermits, additionalNumberOfMessages); + } + readMoreEntries(); + } + + /** + * We should not call readMoreEntries() recursively in the same thread as there is a risk of StackOverflowError. + * + */ + public void readMoreEntriesAsync() { + topic.getBrokerService().executor().execute(this::readMoreEntries); + } + + public synchronized void readMoreEntries() { + if (cursor.isClosed()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor is already closed, skipping read more entries.", cursor.getName()); + } + return; + } + if (isSendInProgress()) { + // we cannot read more entries while sending the previous batch + // otherwise we could re-read the same entries and send duplicates + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic, Due to sending in-progress.", + topic.getName(), getSubscriptionName()); + } + return; + } + if (shouldPauseDeliveryForDelayTracker()) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic, Due to pause delivery for delay tracker.", + topic.getName(), getSubscriptionName()); + } + return; + } + if (topic.isTransferring()) { + // Do not deliver messages for topics that are undergoing transfer, as the acknowledgments would be ignored. + return; + } + + // totalAvailablePermits may be updated by other threads + int firstAvailableConsumerPermits = getFirstAvailableConsumerPermits(); + int currentTotalAvailablePermits = Math.max(totalAvailablePermits, firstAvailableConsumerPermits); + if (currentTotalAvailablePermits > 0 && firstAvailableConsumerPermits > 0) { + Pair calculateResult = calculateToRead(currentTotalAvailablePermits); + int messagesToRead = calculateResult.getLeft(); + long bytesToRead = calculateResult.getRight(); + + if (messagesToRead == -1 || bytesToRead == -1) { + // Skip read as topic/dispatcher has exceed the dispatch rate or previous pending read hasn't complete. + return; + } + + NavigableSet messagesToReplayNow = getMessagesToReplayNow(messagesToRead); + NavigableSet messagesToReplayFiltered = filterOutEntriesWillBeDiscarded(messagesToReplayNow); + if (!messagesToReplayFiltered.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Schedule replay of {} messages for {} consumers", name, + messagesToReplayFiltered.size(), consumerList.size()); + } + + havePendingReplayRead = true; + minReplayedPosition = messagesToReplayNow.first(); + Set deletedMessages = topic.isDelayedDeliveryEnabled() + ? asyncReplayEntriesInOrder(messagesToReplayFiltered) + : asyncReplayEntries(messagesToReplayFiltered); + // clear already acked positions from replay bucket + + deletedMessages.forEach(position -> redeliveryMessages.remove(((Position) position).getLedgerId(), + ((Position) position).getEntryId())); + // if all the entries are acked-entries and cleared up from redeliveryMessages, try to read + // next entries as readCompletedEntries-callback was never called + if ((messagesToReplayFiltered.size() - deletedMessages.size()) == 0) { + havePendingReplayRead = false; + readMoreEntriesAsync(); + } + } else if (BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.get(this) == TRUE) { + if (log.isDebugEnabled()) { + log.debug("[{}] Dispatcher read is blocked due to unackMessages {} reached to max {}", name, + totalUnackedMessages, topic.getMaxUnackedMessagesOnSubscription()); + } + } else if (!havePendingRead && hasConsumersNeededNormalRead()) { + if (shouldPauseOnAckStatePersist(ReadType.Normal)) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic, Due to blocked on ack state persistent.", + topic.getName(), getSubscriptionName()); + } + return; + } + if (log.isDebugEnabled()) { + log.debug("[{}] Schedule read of {} messages for {} consumers", name, messagesToRead, + consumerList.size()); + } + havePendingRead = true; + NavigableSet toReplay = getMessagesToReplayNow(1); + if (!toReplay.isEmpty()) { + minReplayedPosition = toReplay.first(); + redeliveryMessages.add(minReplayedPosition.getLedgerId(), minReplayedPosition.getEntryId()); + } else { + minReplayedPosition = null; + } + + // Filter out and skip read delayed messages exist in DelayedDeliveryTracker + if (delayedDeliveryTracker.isPresent()) { + Predicate skipCondition = null; + final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get(); + if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { + skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker) + .containsMessage(position.getLedgerId(), position.getEntryId()); + } + cursor.asyncReadEntriesWithSkipOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, + topic.getMaxReadPosition(), skipCondition); + } else { + cursor.asyncReadEntriesOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, + topic.getMaxReadPosition()); + } + } else { + if (log.isDebugEnabled()) { + if (!messagesToReplayNow.isEmpty()) { + log.debug("[{}] [{}] Skipping read for the topic: because all entries in replay queue were" + + " filtered out due to the mechanism of Key_Shared mode, and the left consumers have" + + " no permits now", + topic.getName(), getSubscriptionName()); + } else { + log.debug("[{}] Cannot schedule next read until previous one is done", name); + } + } + } + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Consumer buffer is full, pause reading", name); + } + } + } + + private boolean shouldPauseOnAckStatePersist(ReadType readType) { + // Allows new consumers to consume redelivered messages caused by the just-closed consumer. + if (readType != ReadType.Normal) { + return false; + } + if (!((PersistentTopic) subscription.getTopic()).isDispatcherPauseOnAckStatePersistentEnabled()) { + return false; + } + if (cursor == null) { + return true; + } + return blockedDispatcherOnCursorDataCanNotFullyPersist == TRUE; + } + + @Override + protected void reScheduleRead() { + if (isRescheduleReadInProgress.compareAndSet(false, true)) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Reschedule message read in {} ms", topic.getName(), name, MESSAGE_RATE_BACKOFF_MS); + } + topic.getBrokerService().executor().schedule( + () -> { + isRescheduleReadInProgress.set(false); + readMoreEntries(); + }, + MESSAGE_RATE_BACKOFF_MS, TimeUnit.MILLISECONDS); + } + } + + // left pair is messagesToRead, right pair is bytesToRead + protected Pair calculateToRead(int currentTotalAvailablePermits) { + int messagesToRead = Math.min(currentTotalAvailablePermits, readBatchSize); + long bytesToRead = serviceConfig.getDispatcherMaxReadSizeBytes(); + + Consumer c = getRandomConsumer(); + // if turn on precise dispatcher flow control, adjust the record to read + if (c != null && c.isPreciseDispatcherFlowControl()) { + int avgMessagesPerEntry = Math.max(1, c.getAvgMessagesPerEntry()); + messagesToRead = Math.min( + (int) Math.ceil(currentTotalAvailablePermits * 1.0 / avgMessagesPerEntry), + readBatchSize); + } + + if (!isConsumerWritable()) { + // If the connection is not currently writable, we issue the read request anyway, but for a single + // message. The intent here is to keep use the request as a notification mechanism while avoiding to + // read and dispatch a big batch of messages which will need to wait before getting written to the + // socket. + messagesToRead = 1; + } + + // throttle only if: (1) cursor is not active (or flag for throttle-nonBacklogConsumer is enabled) bcz + // active-cursor reads message from cache rather from bookkeeper (2) if topic has reached message-rate + // threshold: then schedule the read after MESSAGE_RATE_BACKOFF_MS + if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { + if (topic.getBrokerDispatchRateLimiter().isPresent()) { + DispatchRateLimiter brokerRateLimiter = topic.getBrokerDispatchRateLimiter().get(); + Pair calculateToRead = + updateMessagesToRead(brokerRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] message-read exceeded broker message-rate {}/{}, schedule after a {}", name, + brokerRateLimiter.getDispatchRateOnMsg(), brokerRateLimiter.getDispatchRateOnByte(), + MESSAGE_RATE_BACKOFF_MS); + } + reScheduleRead(); + return Pair.of(-1, -1L); + } + } + + if (topic.getDispatchRateLimiter().isPresent()) { + DispatchRateLimiter topicRateLimiter = topic.getDispatchRateLimiter().get(); + Pair calculateToRead = + updateMessagesToRead(topicRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] message-read exceeded topic message-rate {}/{}, schedule after a {}", name, + topicRateLimiter.getDispatchRateOnMsg(), topicRateLimiter.getDispatchRateOnByte(), + MESSAGE_RATE_BACKOFF_MS); + } + reScheduleRead(); + return Pair.of(-1, -1L); + } + } + + if (dispatchRateLimiter.isPresent()) { + Pair calculateToRead = + updateMessagesToRead(dispatchRateLimiter.get(), messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] message-read exceeded subscription message-rate {}/{}, schedule after a {}", + name, dispatchRateLimiter.get().getDispatchRateOnMsg(), + dispatchRateLimiter.get().getDispatchRateOnByte(), + MESSAGE_RATE_BACKOFF_MS); + } + reScheduleRead(); + return Pair.of(-1, -1L); + } + } + } + + if (havePendingReplayRead) { + if (log.isDebugEnabled()) { + log.debug("[{}] Skipping replay while awaiting previous read to complete", name); + } + return Pair.of(-1, -1L); + } + + // If messagesToRead is 0 or less, correct it to 1 to prevent IllegalArgumentException + messagesToRead = Math.max(messagesToRead, 1); + bytesToRead = Math.max(bytesToRead, 1); + return Pair.of(messagesToRead, bytesToRead); + } + + protected Set asyncReplayEntries(Set positions) { + return cursor.asyncReplayEntries(positions, this, ReadType.Replay); + } + + protected Set asyncReplayEntriesInOrder(Set positions) { + return cursor.asyncReplayEntries(positions, this, ReadType.Replay, true); + } + + @Override + public boolean isConsumerConnected() { + return !consumerList.isEmpty(); + } + + @Override + public CopyOnWriteArrayList getConsumers() { + return consumerList; + } + + @Override + public synchronized boolean canUnsubscribe(Consumer consumer) { + return consumerList.size() == 1 && consumerSet.contains(consumer); + } + + @Override + public CompletableFuture close(boolean disconnectConsumers, + Optional assignedBrokerLookupData) { + IS_CLOSED_UPDATER.set(this, TRUE); + + Optional delayedDeliveryTracker; + synchronized (this) { + delayedDeliveryTracker = this.delayedDeliveryTracker; + this.delayedDeliveryTracker = Optional.empty(); + } + + delayedDeliveryTracker.ifPresent(DelayedDeliveryTracker::close); + dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); + + return disconnectConsumers + ? disconnectAllConsumers(false, assignedBrokerLookupData) : CompletableFuture.completedFuture(null); + } + + @Override + public synchronized CompletableFuture disconnectAllConsumers( + boolean isResetCursor, Optional assignedBrokerLookupData) { + closeFuture = new CompletableFuture<>(); + if (consumerList.isEmpty()) { + closeFuture.complete(null); + } else { + // Iterator of CopyOnWriteArrayList uses the internal array to do the for-each, and CopyOnWriteArrayList + // will create a new internal array when adding/removing a new item. So remove items in the for-each + // block is safety when the for-each and add/remove are using a same lock. + consumerList.forEach(consumer -> consumer.disconnect(isResetCursor, assignedBrokerLookupData)); + cancelPendingRead(); + } + return closeFuture; + } + + @Override + protected void cancelPendingRead() { + if ((havePendingRead || havePendingReplayRead) && cursor.cancelPendingReadRequest()) { + havePendingRead = false; + havePendingReplayRead = false; + } + } + + @Override + public CompletableFuture disconnectActiveConsumers(boolean isResetCursor) { + return disconnectAllConsumers(isResetCursor); + } + + @Override + public synchronized void resetCloseFuture() { + closeFuture = null; + } + + @Override + public void reset() { + resetCloseFuture(); + IS_CLOSED_UPDATER.set(this, FALSE); + } + + @Override + public SubType getType() { + return SubType.Shared; + } + + @Override + public final synchronized void readEntriesComplete(List entries, Object ctx) { + ReadType readType = (ReadType) ctx; + if (readType == ReadType.Normal) { + havePendingRead = false; + } else { + havePendingReplayRead = false; + } + + if (readBatchSize < serviceConfig.getDispatcherMaxReadBatchSize()) { + int newReadBatchSize = Math.min(readBatchSize * 2, serviceConfig.getDispatcherMaxReadBatchSize()); + if (log.isDebugEnabled()) { + log.debug("[{}] Increasing read batch size from {} to {}", name, readBatchSize, newReadBatchSize); + } + + readBatchSize = newReadBatchSize; + } + + readFailureBackoff.reduceToHalf(); + + if (shouldRewindBeforeReadingOrReplaying && readType == ReadType.Normal) { + // All consumers got disconnected before the completion of the read operation + entries.forEach(Entry::release); + cursor.rewind(); + shouldRewindBeforeReadingOrReplaying = false; + readMoreEntries(); + return; + } + + if (log.isDebugEnabled()) { + log.debug("[{}] Distributing {} messages to {} consumers", name, entries.size(), consumerList.size()); + } + + long size = entries.stream().mapToLong(Entry::getLength).sum(); + updatePendingBytesToDispatch(size); + + // dispatch messages to a separate thread, but still in order for this subscription + // sendMessagesToConsumers is responsible for running broker-side filters + // that may be quite expensive + if (serviceConfig.isDispatcherDispatchMessagesInSubscriptionThread()) { + // setting sendInProgress here, because sendMessagesToConsumers will be executed + // in a separate thread, and we want to prevent more reads + acquireSendInProgress(); + dispatchMessagesThread.execute(() -> { + if (sendMessagesToConsumers(readType, entries, false)) { + updatePendingBytesToDispatch(-size); + readMoreEntries(); + } else { + updatePendingBytesToDispatch(-size); + } + }); + } else { + if (sendMessagesToConsumers(readType, entries, true)) { + updatePendingBytesToDispatch(-size); + readMoreEntriesAsync(); + } else { + updatePendingBytesToDispatch(-size); + } + } + } + + protected synchronized void acquireSendInProgress() { + sendInProgress = true; + } + + protected synchronized void releaseSendInProgress() { + sendInProgress = false; + } + + protected synchronized boolean isSendInProgress() { + return sendInProgress; + } + + protected final synchronized boolean sendMessagesToConsumers(ReadType readType, List entries, + boolean needAcquireSendInProgress) { + if (needAcquireSendInProgress) { + acquireSendInProgress(); + } + try { + return trySendMessagesToConsumers(readType, entries); + } finally { + releaseSendInProgress(); + } + } + + /** + * Dispatch the messages to the Consumers. + * @return true if you want to trigger a new read. + * This method is overridden by other classes, please take a look to other implementations + * if you need to change it. + */ + protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { + if (needTrimAckedMessages()) { + cursor.trimDeletedEntries(entries); + } + + int entriesToDispatch = entries.size(); + // Trigger read more messages + if (entriesToDispatch == 0) { + return true; + } + final MessageMetadata[] metadataArray = new MessageMetadata[entries.size()]; + int remainingMessages = 0; + boolean hasChunk = false; + for (int i = 0; i < metadataArray.length; i++) { + final MessageMetadata metadata = Commands.peekAndCopyMessageMetadata( + entries.get(i).getDataBuffer(), subscription.toString(), -1); + if (metadata != null) { + remainingMessages += metadata.getNumMessagesInBatch(); + if (!hasChunk && metadata.hasUuid()) { + hasChunk = true; + } + } + metadataArray[i] = metadata; + } + if (hasChunk) { + return sendChunkedMessagesToConsumers(readType, entries, metadataArray); + } + + int start = 0; + long totalMessagesSent = 0; + long totalBytesSent = 0; + long totalEntries = 0; + int avgBatchSizePerMsg = remainingMessages > 0 ? Math.max(remainingMessages / entries.size(), 1) : 1; + + // If the dispatcher is closed, firstAvailableConsumerPermits will be 0, which skips dispatching the + // messages. + while (entriesToDispatch > 0 && isAtleastOneConsumerAvailable()) { + Consumer c = getNextConsumer(); + if (c == null) { + // Do nothing, cursor will be rewind at reconnection + log.info("[{}] rewind because no available consumer found from total {}", name, consumerList.size()); + entries.subList(start, entries.size()).forEach(Entry::release); + cursor.rewind(); + return false; + } + + // round-robin dispatch batch size for this consumer + int availablePermits = c.isWritable() ? c.getAvailablePermits() : 1; + if (c.getMaxUnackedMessages() > 0) { + // Avoid negative number + int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + availablePermits = Math.min(availablePermits, remainUnAckedMessages); + } + if (log.isDebugEnabled() && !c.isWritable()) { + log.debug("[{}-{}] consumer is not writable. dispatching only 1 message to {}; " + + "availablePermits are {}", topic.getName(), name, + c, c.getAvailablePermits()); + } + + int messagesForC = Math.min(Math.min(remainingMessages, availablePermits), + serviceConfig.getDispatcherMaxRoundRobinBatchSize()); + messagesForC = Math.max(messagesForC / avgBatchSizePerMsg, 1); + + int end = Math.min(start + messagesForC, entries.size()); + List entriesForThisConsumer = entries.subList(start, end); + + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { + entriesForThisConsumer.forEach(entry -> { + redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); + }); + } + + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + + EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesForThisConsumer.size()); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(entriesForThisConsumer.size()); + totalEntries += filterEntriesForConsumer(metadataArray, start, + entriesForThisConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, + readType == ReadType.Replay, c); + + c.sendMessages(entriesForThisConsumer, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), + sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), redeliveryTracker); + + int msgSent = sendMessageInfo.getTotalMessages(); + remainingMessages -= msgSent; + start += messagesForC; + entriesToDispatch -= messagesForC; + TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, + -(msgSent - batchIndexesAcks.getTotalAckedIndexCount())); + if (log.isDebugEnabled()) { + log.debug("[{}] Added -({} minus {}) permits to TOTAL_AVAILABLE_PERMITS_UPDATER in " + + "PersistentDispatcherMultipleConsumers", + name, msgSent, batchIndexesAcks.getTotalAckedIndexCount()); + } + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); + } + + acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); + + if (entriesToDispatch > 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] No consumers found with available permits, storing {} positions for later replay", name, + entries.size() - start); + } + entries.subList(start, entries.size()).forEach(entry -> { + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + }); + } + return true; + } + + private boolean sendChunkedMessagesToConsumers(ReadType readType, + List entries, + MessageMetadata[] metadataArray) { + final List originalEntryAndMetadataList = new ArrayList<>(metadataArray.length); + for (int i = 0; i < metadataArray.length; i++) { + originalEntryAndMetadataList.add(EntryAndMetadata.create(entries.get(i), metadataArray[i])); + } + + final Map> assignResult = + assignor.assign(originalEntryAndMetadataList, consumerList.size()); + long totalMessagesSent = 0; + long totalBytesSent = 0; + long totalEntries = 0; + final AtomicInteger numConsumers = new AtomicInteger(assignResult.size()); + for (Map.Entry> current : assignResult.entrySet()) { + final Consumer consumer = current.getKey(); + final List entryAndMetadataList = current.getValue(); + final int messagesForC = Math.min(consumer.getAvailablePermits(), entryAndMetadataList.size()); + if (log.isDebugEnabled()) { + log.debug("[{}] select consumer {} with messages num {}, read type is {}", + name, consumer.consumerName(), messagesForC, readType); + } + if (messagesForC < entryAndMetadataList.size()) { + for (int i = messagesForC; i < entryAndMetadataList.size(); i++) { + final EntryAndMetadata entry = entryAndMetadataList.get(i); + addMessageToReplay(entry); + entryAndMetadataList.set(i, null); + } + } + if (messagesForC == 0) { + numConsumers.decrementAndGet(); + continue; + } + if (readType == ReadType.Replay) { + entryAndMetadataList.stream().limit(messagesForC) + .forEach(e -> redeliveryMessages.remove(e.getLedgerId(), e.getEntryId())); + } + final SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + final EntryBatchSizes batchSizes = EntryBatchSizes.get(messagesForC); + final EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); + + totalEntries += filterEntriesForConsumer(entryAndMetadataList, batchSizes, sendMessageInfo, + batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + consumer.sendMessages(entryAndMetadataList, batchSizes, batchIndexesAcks, + sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), + sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker() + ).addListener(future -> { + if (future.isDone() && numConsumers.decrementAndGet() == 0) { + readMoreEntries(); + } + }); + + TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, + -(sendMessageInfo.getTotalMessages() - batchIndexesAcks.getTotalAckedIndexCount())); + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); + } + + acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); + + return numConsumers.get() == 0; // trigger a new readMoreEntries() call + } + + @Override + public synchronized void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + + ReadType readType = (ReadType) ctx; + long waitTimeMillis = readFailureBackoff.next(); + + // Do not keep reading more entries if the cursor is already closed. + if (exception instanceof ManagedLedgerException.CursorAlreadyClosedException) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor is already closed, skipping read more entries", cursor.getName()); + } + // Set the wait time to -1 to avoid rescheduling the read. + waitTimeMillis = -1; + } else if (exception instanceof NoMoreEntriesToReadException) { + if (cursor.getNumberOfEntriesInBacklog(false) == 0) { + // Topic has been terminated and there are no more entries to read + // Notify the consumer only if all the messages were already acknowledged + checkAndApplyReachedEndOfTopicOrTopicMigration(consumerList); + } + } else if (exception.getCause() instanceof TransactionBufferException.TransactionNotSealedException + || exception.getCause() instanceof ManagedLedgerException.OffloadReadHandleClosedException) { + waitTimeMillis = 1; + if (log.isDebugEnabled()) { + log.debug("[{}] Error reading transaction entries : {}, Read Type {} - Retrying to read in {} seconds", + name, exception.getMessage(), readType, waitTimeMillis / 1000.0); + } + } else if (!(exception instanceof TooManyRequestsException)) { + log.error("[{}] Error reading entries at {} : {}, Read Type {} - Retrying to read in {} seconds", name, + cursor.getReadPosition(), exception.getMessage(), readType, waitTimeMillis / 1000.0); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Error reading entries at {} : {}, Read Type {} - Retrying to read in {} seconds", name, + cursor.getReadPosition(), exception.getMessage(), readType, waitTimeMillis / 1000.0); + } + } + + if (shouldRewindBeforeReadingOrReplaying) { + shouldRewindBeforeReadingOrReplaying = false; + cursor.rewind(); + } + + if (readType == ReadType.Normal) { + havePendingRead = false; + } else { + havePendingReplayRead = false; + if (exception instanceof ManagedLedgerException.InvalidReplayPositionException) { + Position markDeletePosition = (Position) cursor.getMarkDeletedPosition(); + redeliveryMessages.removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); + } + } + + readBatchSize = serviceConfig.getDispatcherMinReadBatchSize(); + // Skip read if the waitTimeMillis is a nagetive value. + if (waitTimeMillis >= 0) { + scheduleReadEntriesWithDelay(exception, readType, waitTimeMillis); + } + } + + @VisibleForTesting + void scheduleReadEntriesWithDelay(Exception e, ReadType readType, long waitTimeMillis) { + topic.getBrokerService().executor().schedule(() -> { + synchronized (PersistentDispatcherMultipleConsumersClassic.this) { + // If it's a replay read we need to retry even if there's already + // another scheduled read, otherwise we'd be stuck until + // more messages are published. + if (!havePendingRead || readType == ReadType.Replay) { + log.info("[{}] Retrying read operation", name); + readMoreEntries(); + } else { + log.info("[{}] Skipping read retry: havePendingRead {}", name, havePendingRead, e); + } + } + }, waitTimeMillis, TimeUnit.MILLISECONDS); + } + + private boolean needTrimAckedMessages() { + if (lastIndividualDeletedRangeFromCursorRecovery == null) { + return false; + } else { + return lastIndividualDeletedRangeFromCursorRecovery.upperEndpoint() + .compareTo((Position) cursor.getReadPosition()) > 0; + } + } + + /** + * returns true only if {@link AbstractDispatcherMultipleConsumers#consumerList} + * has atleast one unblocked consumer and have available permits. + * + * @return + */ + protected boolean isAtleastOneConsumerAvailable() { + return getFirstAvailableConsumerPermits() > 0; + } + + protected int getFirstAvailableConsumerPermits() { + if (consumerList.isEmpty() || IS_CLOSED_UPDATER.get(this) == TRUE) { + // abort read if no consumers are connected or if disconnect is initiated + return 0; + } + for (Consumer consumer : consumerList) { + if (consumer != null && !consumer.isBlocked()) { + int availablePermits = consumer.getAvailablePermits(); + if (availablePermits > 0) { + return availablePermits; + } + } + } + return 0; + } + + private boolean isConsumerWritable() { + for (Consumer consumer : consumerList) { + if (consumer.isWritable()) { + return true; + } + } + if (log.isDebugEnabled()) { + log.debug("[{}-{}] consumer is not writable", topic.getName(), name); + } + return false; + } + + @Override + public boolean isConsumerAvailable(Consumer consumer) { + return consumer != null && !consumer.isBlocked() && consumer.getAvailablePermits() > 0; + } + + @Override + public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch) { + consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + if (addMessageToReplay(ledgerId, entryId, stickyKeyHash)) { + redeliveryTracker.incrementAndGetRedeliveryCount((PositionFactory.create(ledgerId, entryId))); + } + }); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, + redeliveryMessages); + } + readMoreEntries(); + } + + @Override + public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + positions.forEach(position -> { + // TODO: We want to pass a sticky key hash as a third argument to guarantee the order of the messages + // on Key_Shared subscription, but it's difficult to get the sticky key here + if (addMessageToReplay(position.getLedgerId(), position.getEntryId())) { + redeliveryTracker.incrementAndGetRedeliveryCount(position); + } + }); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions); + } + readMoreEntries(); + } + + @Override + public void addUnAckedMessages(int numberOfMessages) { + int maxUnackedMessages = topic.getMaxUnackedMessagesOnSubscription(); + // don't block dispatching if maxUnackedMessages = 0 + if (maxUnackedMessages <= 0 && blockedDispatcherOnUnackedMsgs == TRUE + && BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, TRUE, FALSE)) { + log.info("[{}] Dispatcher is unblocked, since maxUnackedMessagesPerSubscription=0", name); + readMoreEntriesAsync(); + } + + int unAckedMessages = TOTAL_UNACKED_MESSAGES_UPDATER.addAndGet(this, numberOfMessages); + if (unAckedMessages >= maxUnackedMessages && maxUnackedMessages > 0 + && BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, FALSE, TRUE)) { + // block dispatcher if it reaches maxUnAckMsg limit + log.debug("[{}] Dispatcher is blocked due to unackMessages {} reached to max {}", name, + unAckedMessages, maxUnackedMessages); + } else if (topic.getBrokerService().isBrokerDispatchingBlocked() + && blockedDispatcherOnUnackedMsgs == TRUE) { + // unblock dispatcher: if dispatcher is blocked due to broker-unackMsg limit and if it ack back enough + // messages + if (totalUnackedMessages < (topic.getBrokerService().maxUnackedMsgsPerDispatcher / 2)) { + if (BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, TRUE, FALSE)) { + // it removes dispatcher from blocked list and unblocks dispatcher by scheduling read + topic.getBrokerService().unblockDispatchersOnUnAckMessages(Lists.newArrayList(this)); + } + } + } else if (blockedDispatcherOnUnackedMsgs == TRUE && unAckedMessages < maxUnackedMessages / 2) { + // unblock dispatcher if it acks back enough messages + if (BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, TRUE, FALSE)) { + log.debug("[{}] Dispatcher is unblocked", name); + readMoreEntriesAsync(); + } + } + // increment broker-level count + topic.getBrokerService().addUnAckedMessages(this, numberOfMessages); + } + + @Override + public void afterAckMessages(Throwable exOfDeletion, Object ctxOfDeletion) { + boolean unPaused = blockedDispatcherOnCursorDataCanNotFullyPersist == FALSE; + // Trigger a new read if needed. + boolean shouldPauseNow = !checkAndResumeIfPaused(); + // Switch stat to "paused" if needed. + if (unPaused && shouldPauseNow) { + if (!BLOCKED_DISPATCHER_ON_CURSOR_DATA_CAN_NOT_FULLY_PERSIST_UPDATER + .compareAndSet(this, FALSE, TRUE)) { + // Retry due to conflict update. + afterAckMessages(exOfDeletion, ctxOfDeletion); + } + } + } + + @Override + public boolean checkAndResumeIfPaused() { + boolean paused = blockedDispatcherOnCursorDataCanNotFullyPersist == TRUE; + // Calling "cursor.isCursorDataFullyPersistable()" will loop the collection "individualDeletedMessages". It is + // not a light method. + // If never enabled "dispatcherPauseOnAckStatePersistentEnabled", skip the following checks to improve + // performance. + if (!paused && !topic.isDispatcherPauseOnAckStatePersistentEnabled()){ + // "true" means no need to pause. + return true; + } + // Enabled "dispatcherPauseOnAckStatePersistentEnabled" before. + boolean shouldPauseNow = !cursor.isCursorDataFullyPersistable() + && topic.isDispatcherPauseOnAckStatePersistentEnabled(); + // No need to change. + if (paused == shouldPauseNow) { + return !shouldPauseNow; + } + // Should change to "un-pause". + if (paused && !shouldPauseNow) { + // If there was no previous pause due to cursor data is too large to persist, we don't need to manually + // trigger a new read. This can avoid too many CPU circles. + if (BLOCKED_DISPATCHER_ON_CURSOR_DATA_CAN_NOT_FULLY_PERSIST_UPDATER.compareAndSet(this, TRUE, FALSE)) { + readMoreEntriesAsync(); + } else { + // Retry due to conflict update. + checkAndResumeIfPaused(); + } + } + return !shouldPauseNow; + } + + public boolean isBlockedDispatcherOnUnackedMsgs() { + return blockedDispatcherOnUnackedMsgs == TRUE; + } + + public void blockDispatcherOnUnackedMsgs() { + blockedDispatcherOnUnackedMsgs = TRUE; + } + + public void unBlockDispatcherOnUnackedMsgs() { + blockedDispatcherOnUnackedMsgs = FALSE; + } + + public int getTotalUnackedMessages() { + return totalUnackedMessages; + } + + public String getName() { + return name; + } + + @Override + public RedeliveryTracker getRedeliveryTracker() { + return redeliveryTracker; + } + + @Override + public Optional getRateLimiter() { + return dispatchRateLimiter; + } + + @Override + public boolean initializeDispatchRateLimiterIfNeeded() { + if (!dispatchRateLimiter.isPresent() && DispatchRateLimiter.isDispatchRateEnabled( + topic.getSubscriptionDispatchRate(getSubscriptionName()))) { + this.dispatchRateLimiter = + Optional.of(new DispatchRateLimiter(topic, getSubscriptionName(), Type.SUBSCRIPTION)); + return true; + } + return false; + } + + @Override + public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata msgMetadata) { + if (!topic.isDelayedDeliveryEnabled()) { + // If broker has the feature disabled, always deliver messages immediately + return false; + } + + synchronized (this) { + 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.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis()); + + long deliverAtTime = msgMetadata.hasDeliverAtTime() ? msgMetadata.getDeliverAtTime() : -1L; + return delayedDeliveryTracker.get().addMessage(ledgerId, entryId, deliverAtTime); + } + } + + protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + if (delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().hasMessageAvailable()) { + delayedDeliveryTracker.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis()); + NavigableSet messagesAvailableNow = + delayedDeliveryTracker.get().getScheduledMessages(maxMessagesToRead); + messagesAvailableNow.forEach(p -> redeliveryMessages.add(p.getLedgerId(), p.getEntryId())); + } + + if (!redeliveryMessages.isEmpty()) { + return redeliveryMessages.getMessagesToReplayNow(maxMessagesToRead, position -> true); + } else { + return Collections.emptyNavigableSet(); + } + } + + /** + * This is a mode method designed for Key_Shared mode. + * Filter out the entries that will be discarded due to the order guarantee mechanism of Key_Shared mode. + * This method is in order to avoid the scenario below: + * - Get positions from the Replay queue. + * - Read entries from BK. + * - The order guarantee mechanism of Key_Shared mode filtered out all the entries. + * - Delivery non entry to the client, but we did a BK read. + */ + protected NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + return src; + } + + /** + * This is a mode method designed for Key_Shared mode, to avoid unnecessary stuck. + * See detail {@link PersistentStickyKeyDispatcherMultipleConsumersClassic#hasConsumersNeededNormalRead}. + */ + protected boolean hasConsumersNeededNormalRead() { + return true; + } + + protected synchronized boolean shouldPauseDeliveryForDelayTracker() { + return delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().shouldPauseAllDeliveries(); + } + + @Override + public long getNumberOfDelayedMessages() { + return delayedDeliveryTracker.map(DelayedDeliveryTracker::getNumberOfDelayedMessages).orElse(0L); + } + + @Override + public CompletableFuture clearDelayedMessages() { + if (!topic.isDelayedDeliveryEnabled()) { + return CompletableFuture.completedFuture(null); + } + + if (delayedDeliveryTracker.isPresent()) { + return this.delayedDeliveryTracker.get().clear(); + } else { + DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory = + topic.getBrokerService().getDelayedDeliveryTrackerFactory(); + if (delayedDeliveryTrackerFactory instanceof BucketDelayedDeliveryTrackerFactory + bucketDelayedDeliveryTrackerFactory) { + return bucketDelayedDeliveryTrackerFactory.cleanResidualSnapshots(cursor); + } + return CompletableFuture.completedFuture(null); + } + } + + @Override + public void cursorIsReset() { + if (this.lastIndividualDeletedRangeFromCursorRecovery != null) { + this.lastIndividualDeletedRangeFromCursorRecovery = null; + } + } + + private void addMessageToReplay(Entry entry) { + addMessageToReplay(entry.getLedgerId(), entry.getEntryId()); + entry.release(); + } + + protected boolean addMessageToReplay(long ledgerId, long entryId, long stickyKeyHash) { + if (checkIfMessageIsUnacked(ledgerId, entryId)) { + redeliveryMessages.add(ledgerId, entryId, stickyKeyHash); + return true; + } else { + return false; + } + } + + protected boolean addMessageToReplay(long ledgerId, long entryId) { + if (checkIfMessageIsUnacked(ledgerId, entryId)) { + redeliveryMessages.add(ledgerId, entryId); + return true; + } else { + return false; + } + } + + private boolean checkIfMessageIsUnacked(long ledgerId, long entryId) { + Position markDeletePosition = cursor.getMarkDeletedPosition(); + return (markDeletePosition == null || ledgerId > markDeletePosition.getLedgerId() + || (ledgerId == markDeletePosition.getLedgerId() && entryId > markDeletePosition.getEntryId())); + } + + @Override + public boolean checkAndUnblockIfStuck() { + if (cursor.checkAndUpdateReadPositionChanged()) { + return false; + } + // consider dispatch is stuck if : dispatcher has backlog, available-permits and there is no pending read + if (totalAvailablePermits > 0 && !havePendingReplayRead && !havePendingRead + && cursor.getNumberOfEntriesInBacklog(false) > 0) { + log.warn("{}-{} Dispatcher is stuck and unblocking by issuing reads", topic.getName(), name); + readMoreEntries(); + return true; + } + return false; + } + + public PersistentTopic getTopic() { + return topic; + } + + + public long getDelayedTrackerMemoryUsage() { + return delayedDeliveryTracker.map(DelayedDeliveryTracker::getBufferMemoryUsage).orElse(0L); + } + + public Map getBucketDelayedIndexStats() { + if (delayedDeliveryTracker.isEmpty()) { + return Collections.emptyMap(); + } + + if (delayedDeliveryTracker.get() instanceof BucketDelayedDeliveryTracker) { + return ((BucketDelayedDeliveryTracker) delayedDeliveryTracker.get()).genTopicMetricMap(); + } + + return Collections.emptyMap(); + } + + @Override + public boolean isClassic() { + return true; + } + + public ManagedCursor getCursor() { + return cursor; + } + + protected int getStickyKeyHash(Entry entry) { + return StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; + } + + public Subscription getSubscription() { + return subscription; + } + + public long getNumberOfMessagesInReplay() { + return redeliveryMessages.size(); + } + + @Override + public boolean isHavePendingRead() { + return havePendingRead; + } + + @Override + public boolean isHavePendingReplayRead() { + return havePendingReplayRead; + } + + private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumersClassic.class); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 925e99ed699a2..df053e6d8a549 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -50,6 +50,7 @@ import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; @@ -60,7 +61,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDispatcherMultipleConsumers { +public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDispatcherMultipleConsumers implements + StickyKeyDispatcher { private final boolean allowOutOfOrderDelivery; private final StickyKeyConsumerSelector selector; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java new file mode 100644 index 0000000000000..c227bf5b435bc --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java @@ -0,0 +1,583 @@ +/* + * 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.service.persistent; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.util.concurrent.FastThreadLocal; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.commons.collections4.MapUtils; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.broker.service.ConsistentHashingStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryAndMetadata; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; +import org.apache.pulsar.broker.service.EntryBatchSizes; +import org.apache.pulsar.broker.service.HashRangeAutoSplitStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.HashRangeExclusiveStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.SendMessageInfo; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.api.proto.KeySharedMode; +import org.apache.pulsar.common.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the "classic" dispatcher implementation for the Key_Shared subscription that was used before + * Pulsar 4.0.0 and PIP-379. When `subscriptionKeySharedUseClassicPersistentImplementation=true`, + * this dispatcher will be used. The main purpose is to provide a way to rollback to the old behavior + * in case of issues with the preferred implementation. + */ +public class PersistentStickyKeyDispatcherMultipleConsumersClassic + extends PersistentDispatcherMultipleConsumersClassic implements StickyKeyDispatcher { + + private final boolean allowOutOfOrderDelivery; + private final StickyKeyConsumerSelector selector; + + private boolean isDispatcherStuckOnReplays = false; + private final KeySharedMode keySharedMode; + + /** + * When a consumer joins, it will be added to this map with the current read position. + * This means that, in order to preserve ordering, new consumers can only receive old + * messages, until the mark-delete position will move past this point. + */ + private final LinkedHashMap recentlyJoinedConsumers; + + PersistentStickyKeyDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, + Subscription subscription, ServiceConfiguration conf, + KeySharedMeta ksm) { + super(topic, cursor, subscription, ksm.isAllowOutOfOrderDelivery()); + + this.allowOutOfOrderDelivery = ksm.isAllowOutOfOrderDelivery(); + this.recentlyJoinedConsumers = allowOutOfOrderDelivery ? null : new LinkedHashMap<>(); + this.keySharedMode = ksm.getKeySharedMode(); + switch (this.keySharedMode) { + case AUTO_SPLIT: + if (conf.isSubscriptionKeySharedUseConsistentHashing()) { + selector = new ConsistentHashingStickyKeyConsumerSelector( + conf.getSubscriptionKeySharedConsistentHashingReplicaPoints(), + false, + // Classic implementation uses Integer.MAX_VALUE - 1 as the range end value + Integer.MAX_VALUE - 1); + } else { + selector = new HashRangeAutoSplitStickyKeyConsumerSelector(); + } + break; + + case STICKY: + this.selector = new HashRangeExclusiveStickyKeyConsumerSelector(); + break; + + default: + throw new IllegalArgumentException("Invalid key-shared mode: " + keySharedMode); + } + } + + @VisibleForTesting + public StickyKeyConsumerSelector getSelector() { + return selector; + } + + @Override + public synchronized CompletableFuture addConsumer(Consumer consumer) { + if (IS_CLOSED_UPDATER.get(this) == TRUE) { + log.warn("[{}] Dispatcher is already closed. Closing consumer {}", name, consumer); + consumer.disconnect(); + return CompletableFuture.completedFuture(null); + } + return super.addConsumer(consumer).thenCompose(__ -> + selector.addConsumer(consumer).handle((result, ex) -> { + if (ex != null) { + synchronized (PersistentStickyKeyDispatcherMultipleConsumersClassic.this) { + consumerSet.removeAll(consumer); + consumerList.remove(consumer); + } + throw FutureUtil.wrapToCompletionException(ex); + } + return result; + }) + ).thenRun(() -> { + synchronized (PersistentStickyKeyDispatcherMultipleConsumersClassic.this) { + Position readPositionWhenJoining = (Position) cursor.getReadPosition(); + consumer.setReadPositionWhenJoining(readPositionWhenJoining); + // If this was the 1st consumer, or if all the messages are already acked, then we + // don't need to do anything special + if (!allowOutOfOrderDelivery + && recentlyJoinedConsumers != null + && consumerList.size() > 1 + && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) { + recentlyJoinedConsumers.put(consumer, readPositionWhenJoining); + } + } + }); + } + + @Override + public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { + // The consumer must be removed from the selector before calling the superclass removeConsumer method. + // In the superclass removeConsumer method, the pending acks that the consumer has are added to + // redeliveryMessages. If the consumer has not been removed from the selector at this point, + // the broker will try to redeliver the messages to the consumer that has already been closed. + // As a result, the messages are not redelivered to any consumer, and the mark-delete position does not move, + // eventually causing all consumers to get stuck. + selector.removeConsumer(consumer); + super.removeConsumer(consumer); + if (recentlyJoinedConsumers != null) { + recentlyJoinedConsumers.remove(consumer); + if (consumerList.size() == 1) { + recentlyJoinedConsumers.clear(); + } + if (removeConsumersFromRecentJoinedConsumers() || !redeliveryMessages.isEmpty()) { + readMoreEntries(); + } + } + } + + private static final FastThreadLocal>> localGroupedEntries = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + + private static final FastThreadLocal>> localGroupedPositions = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + + @Override + protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { + long totalMessagesSent = 0; + long totalBytesSent = 0; + long totalEntries = 0; + int entriesCount = entries.size(); + + // Trigger read more messages + if (entriesCount == 0) { + return true; + } + + if (consumerSet.isEmpty()) { + entries.forEach(Entry::release); + cursor.rewind(); + return false; + } + + // A corner case that we have to retry a readMoreEntries in order to preserver order delivery. + // This may happen when consumer closed. See issue #12885 for details. + if (!allowOutOfOrderDelivery) { + NavigableSet messagesToReplayNow = this.getMessagesToReplayNow(1); + if (messagesToReplayNow != null && !messagesToReplayNow.isEmpty()) { + Position replayPosition = messagesToReplayNow.first(); + + // We have received a message potentially from the delayed tracker and, since we're not using it + // right now, it needs to be added to the redelivery tracker or we won't attempt anymore to + // resend it (until we disconnect consumer). + redeliveryMessages.add(replayPosition.getLedgerId(), replayPosition.getEntryId()); + + if (this.minReplayedPosition != null) { + // If relayPosition is a new entry wither smaller position is inserted for redelivery during this + // async read, it is possible that this relayPosition should dispatch to consumer first. So in + // order to preserver order delivery, we need to discard this read result, and try to trigger a + // replay read, that containing "relayPosition", by calling readMoreEntries. + if (replayPosition.compareTo(minReplayedPosition) < 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] Position {} (<{}) is inserted for relay during current {} read, " + + "discard this read and retry with readMoreEntries.", + name, replayPosition, minReplayedPosition, readType); + } + if (readType == ReadType.Normal) { + entries.forEach(entry -> { + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + }); + } else if (readType == ReadType.Replay) { + entries.forEach(Entry::release); + } + return true; + } + } + } + } + + final Map> groupedEntries = localGroupedEntries.get(); + groupedEntries.clear(); + final Map> consumerStickyKeyHashesMap = new HashMap<>(); + + for (Entry entry : entries) { + int stickyKeyHash = getStickyKeyHash(entry); + Consumer c = selector.select(stickyKeyHash); + if (c != null) { + groupedEntries.computeIfAbsent(c, k -> new ArrayList<>()).add(entry); + consumerStickyKeyHashesMap.computeIfAbsent(c, k -> new HashSet<>()).add(stickyKeyHash); + } else { + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + } + } + + AtomicInteger keyNumbers = new AtomicInteger(groupedEntries.size()); + + int currentThreadKeyNumber = groupedEntries.size(); + if (currentThreadKeyNumber == 0) { + currentThreadKeyNumber = -1; + } + for (Map.Entry> current : groupedEntries.entrySet()) { + Consumer consumer = current.getKey(); + assert consumer != null; // checked when added to groupedEntries + List entriesWithSameKey = current.getValue(); + int entriesWithSameKeyCount = entriesWithSameKey.size(); + int availablePermits = getAvailablePermits(consumer); + int messagesForC = getRestrictedMaxEntriesForConsumer(consumer, + entriesWithSameKey.stream().map(Entry::getPosition).collect(Collectors.toList()), availablePermits, + readType, consumerStickyKeyHashesMap.get(consumer)); + if (log.isDebugEnabled()) { + log.debug("[{}] select consumer {} with messages num {}, read type is {}", + name, consumer.consumerName(), messagesForC, readType); + } + + if (messagesForC < entriesWithSameKeyCount) { + // We are not able to push all the messages with given key to its consumer, + // so we discard for now and mark them for later redelivery + for (int i = messagesForC; i < entriesWithSameKeyCount; i++) { + Entry entry = entriesWithSameKey.get(i); + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + entriesWithSameKey.set(i, null); + } + } + + if (messagesForC > 0) { + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { + for (int i = 0; i < messagesForC; i++) { + Entry entry = entriesWithSameKey.get(i); + redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); + } + } + + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + EntryBatchSizes batchSizes = EntryBatchSizes.get(messagesForC); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); + totalEntries += filterEntriesForConsumer(entriesWithSameKey, batchSizes, sendMessageInfo, + batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + consumer.sendMessages(entriesWithSameKey, batchSizes, batchIndexesAcks, + sendMessageInfo.getTotalMessages(), + sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), + getRedeliveryTracker()).addListener(future -> { + if (future.isDone() && keyNumbers.decrementAndGet() == 0) { + readMoreEntries(); + } + }); + + TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, + -(sendMessageInfo.getTotalMessages() - batchIndexesAcks.getTotalAckedIndexCount())); + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); + } else { + currentThreadKeyNumber = keyNumbers.decrementAndGet(); + } + } + + // acquire message-dispatch permits for already delivered messages + acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); + + if (totalMessagesSent == 0 && (recentlyJoinedConsumers == null || recentlyJoinedConsumers.isEmpty())) { + // This means, that all the messages we've just read cannot be dispatched right now. + // This condition can only happen when: + // 1. We have consumers ready to accept messages (otherwise the would not haven been triggered) + // 2. All keys in the current set of messages are routing to consumers that are currently busy + // + // The solution here is to move on and read next batch of messages which might hopefully contain + // also keys meant for other consumers. + // + // We do it unless that are "recently joined consumers". In that case, we would be looking + // ahead in the stream while the new consumers are not ready to accept the new messages, + // therefore would be most likely only increase the distance between read-position and mark-delete + // position. + isDispatcherStuckOnReplays = true; + return true; + } else if (currentThreadKeyNumber == 0) { + return true; + } + return false; + } + + private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List entries, + int availablePermits, ReadType readType, Set stickyKeyHashes) { + int maxMessages = Math.min(entries.size(), availablePermits); + if (maxMessages == 0) { + return 0; + } + if (readType == ReadType.Normal && stickyKeyHashes != null + && redeliveryMessages.containsStickyKeyHashes(stickyKeyHashes)) { + // If redeliveryMessages contains messages that correspond to the same hash as the messages + // that the dispatcher is trying to send, do not send those messages for order guarantee + return 0; + } + if (recentlyJoinedConsumers == null) { + return maxMessages; + } + removeConsumersFromRecentJoinedConsumers(); + Position maxReadPosition = recentlyJoinedConsumers.get(consumer); + // At this point, all the old messages were already consumed and this consumer + // is now ready to receive any message + if (maxReadPosition == null) { + // The consumer has not recently joined, so we can send all messages + return maxMessages; + } + + // If the read type is Replay, we should avoid send messages that hold by other consumer to the new consumers, + // For example, we have 10 messages [0,1,2,3,4,5,6,7,8,9] + // If the consumer0 get message 0 and 1, and does not acked message 0, then consumer1 joined, + // when consumer1 get message 2,3, the broker will not dispatch messages to consumer1 + // because of the mark delete position did not move forward. + // So message 2,3 will stored in the redeliver tracker. + // Now, consumer2 joined, it will read new messages from the cursor, + // so the recentJoinedPosition is 4 for consumer2 + // Because of there are messages need to redeliver, so the broker will read the redelivery message first [2,3] + // message [2,3] is lower than the recentJoinedPosition 4, + // so the message [2,3] will dispatched to the consumer2 + // But the message [2,3] should not dispatch to consumer2. + + if (readType == ReadType.Replay) { + Position minReadPositionForRecentJoinedConsumer = recentlyJoinedConsumers.values().iterator().next(); + if (minReadPositionForRecentJoinedConsumer != null + && minReadPositionForRecentJoinedConsumer.compareTo(maxReadPosition) < 0) { + maxReadPosition = minReadPositionForRecentJoinedConsumer; + } + } + // Here, the consumer is one that has recently joined, so we can only send messages that were + // published before it has joined. + for (int i = 0; i < maxMessages; i++) { + if (((Position) entries.get(i)).compareTo(maxReadPosition) >= 0) { + // We have already crossed the divider line. All messages in the list are now + // newer than what we can currently dispatch to this consumer + return i; + } + } + + return maxMessages; + } + + @Override + public void markDeletePositionMoveForward() { + // Execute the notification in different thread to avoid a mutex chain here + // from the delete operation that was completed + topic.getBrokerService().getTopicOrderedExecutor().execute(() -> { + synchronized (PersistentStickyKeyDispatcherMultipleConsumersClassic.this) { + if (recentlyJoinedConsumers != null && !recentlyJoinedConsumers.isEmpty() + && removeConsumersFromRecentJoinedConsumers()) { + // After we process acks, we need to check whether the mark-delete position was advanced and we + // can finally read more messages. It's safe to call readMoreEntries() multiple times. + readMoreEntries(); + } + } + }); + } + + private boolean removeConsumersFromRecentJoinedConsumers() { + if (MapUtils.isEmpty(recentlyJoinedConsumers)) { + return false; + } + Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); + boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; + Position mdp = (Position) cursor.getMarkDeletedPosition(); + if (mdp != null) { + Position nextPositionOfTheMarkDeletePosition = + ((ManagedLedgerImpl) cursor.getManagedLedger()).getNextValidPosition(mdp); + while (itr.hasNext()) { + Map.Entry entry = itr.next(); + if (entry.getValue().compareTo(nextPositionOfTheMarkDeletePosition) <= 0) { + itr.remove(); + hasConsumerRemovedFromTheRecentJoinedConsumers = true; + } else { + break; + } + } + } + return hasConsumerRemovedFromTheRecentJoinedConsumers; + } + + @Override + protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + if (isDispatcherStuckOnReplays) { + // If we're stuck on replay, we want to move forward reading on the topic (until the overall max-unacked + // messages kicks in), instead of keep replaying the same old messages, since the consumer that these + // messages are routing to might be busy at the moment + this.isDispatcherStuckOnReplays = false; + return Collections.emptyNavigableSet(); + } else { + return super.getMessagesToReplayNow(maxMessagesToRead); + } + } + + private int getAvailablePermits(Consumer c) { + int availablePermits = Math.max(c.getAvailablePermits(), 0); + if (c.getMaxUnackedMessages() > 0) { + // Avoid negative number + int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + availablePermits = Math.min(availablePermits, remainUnAckedMessages); + } + return availablePermits; + } + + @Override + protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So skip this filter out. + if (isAllowOutOfOrderDelivery()) { + return src; + } + if (src.isEmpty()) { + return src; + } + NavigableSet res = new TreeSet<>(); + // Group positions. + final Map> groupedPositions = localGroupedPositions.get(); + groupedPositions.clear(); + for (Position pos : src) { + Long stickyKeyHash = redeliveryMessages.getHash(pos.getLedgerId(), pos.getEntryId()); + if (stickyKeyHash == null) { + res.add(pos); + continue; + } + Consumer c = selector.select(stickyKeyHash.intValue()); + if (c == null) { + // Maybe using HashRangeExclusiveStickyKeyConsumerSelector. + continue; + } + groupedPositions.computeIfAbsent(c, k -> new ArrayList<>()).add(pos); + } + // Filter positions by the Recently Joined Position rule. + for (Map.Entry> item : groupedPositions.entrySet()) { + int availablePermits = getAvailablePermits(item.getKey()); + if (availablePermits == 0) { + continue; + } + int posCountToRead = getRestrictedMaxEntriesForConsumer(item.getKey(), item.getValue(), availablePermits, + ReadType.Replay, null); + if (posCountToRead > 0) { + res.addAll(item.getValue().subList(0, posCountToRead)); + } + } + return res; + } + + /** + * In Key_Shared mode, the consumer will not receive any entries from a normal reading if it is included in + * {@link #recentlyJoinedConsumers}, they can only receive entries from replay reads. + * If all entries in {@link #redeliveryMessages} have been filtered out due to the order guarantee mechanism, + * Broker need a normal read to make the consumers not included in @link #recentlyJoinedConsumers} will not be + * stuck. See https://github.com/apache/pulsar/pull/7105. + */ + @Override + protected boolean hasConsumersNeededNormalRead() { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So the method "filterOutEntriesWillBeDiscarded" will filter out nothing, just return "true" here. + if (isAllowOutOfOrderDelivery()) { + return true; + } + for (Consumer consumer : consumerList) { + if (consumer == null || consumer.isBlocked()) { + continue; + } + if (recentlyJoinedConsumers.containsKey(consumer)) { + continue; + } + if (consumer.getAvailablePermits() > 0) { + return true; + } + } + return false; + } + + @Override + public SubType getType() { + return SubType.Key_Shared; + } + + @Override + protected Set asyncReplayEntries(Set positions) { + return cursor.asyncReplayEntries(positions, this, ReadType.Replay, true); + } + + public KeySharedMode getKeySharedMode() { + return this.keySharedMode; + } + + public boolean isAllowOutOfOrderDelivery() { + return this.allowOutOfOrderDelivery; + } + + public boolean hasSameKeySharedPolicy(KeySharedMeta ksm) { + return (ksm.getKeySharedMode() == this.keySharedMode + && ksm.isAllowOutOfOrderDelivery() == this.allowOutOfOrderDelivery); + } + + public LinkedHashMap getRecentlyJoinedConsumers() { + return recentlyJoinedConsumers; + } + + public Map> getConsumerKeyHashRanges() { + return selector.getConsumerKeyHashRanges(); + } + + @Override + protected int getStickyKeyHash(Entry entry) { + if (entry instanceof EntryAndMetadata entryAndMetadata) { + // use the cached sticky key hash if available, otherwise calculate the sticky key hash and cache it + return entryAndMetadata.getOrUpdateCachedStickyKeyHash(selector::makeStickyKeyHash); + } + return selector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer())); + } + + private static final Logger log = + LoggerFactory.getLogger(PersistentStickyKeyDispatcherMultipleConsumersClassic.class); + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index b8d351bddf839..eaa147b81b126 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -25,6 +25,7 @@ import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -69,6 +70,7 @@ import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryFilterSupport; import org.apache.pulsar.broker.service.GetStatsOptions; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.plugin.EntryFilter; @@ -250,7 +252,12 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { case Shared: if (dispatcher == null || dispatcher.getType() != SubType.Shared) { previousDispatcher = dispatcher; - dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, this); + ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); + if (config.isSubscriptionSharedUseClassicPersistentImplementation()) { + dispatcher = new PersistentDispatcherMultipleConsumersClassic(topic, cursor, this); + } else { + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, this); + } } break; case Failover: @@ -271,11 +278,19 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { case Key_Shared: KeySharedMeta ksm = consumer.getKeySharedMeta(); if (dispatcher == null || dispatcher.getType() != SubType.Key_Shared - || !((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher) + || !((StickyKeyDispatcher) dispatcher) .hasSameKeySharedPolicy(ksm)) { previousDispatcher = dispatcher; - dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this, - topic.getBrokerService().getPulsar().getConfiguration(), ksm); + ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); + if (config.isSubscriptionKeySharedUseClassicPersistentImplementation()) { + dispatcher = + new PersistentStickyKeyDispatcherMultipleConsumersClassic(topic, cursor, + this, + topic.getBrokerService().getPulsar().getConfiguration(), ksm); + } else { + dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this, + topic.getBrokerService().getPulsar().getConfiguration(), ksm); + } } break; default: @@ -1221,7 +1236,7 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge Dispatcher dispatcher = this.dispatcher; if (dispatcher != null) { Map> consumerKeyHashRanges = getType() == SubType.Key_Shared - ? ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher).getConsumerKeyHashRanges() : null; + ? ((StickyKeyDispatcher) dispatcher).getConsumerKeyHashRanges() : null; dispatcher.getConsumers().forEach(consumer -> { ConsumerStatsImpl consumerStats = consumer.getStats(); if (!getStatsOptions.isExcludeConsumers()) { @@ -1260,17 +1275,18 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge } } - if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (dispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subStats.delayedMessageIndexSizeInBytes = - ((PersistentDispatcherMultipleConsumers) dispatcher).getDelayedTrackerMemoryUsage(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getDelayedTrackerMemoryUsage(); subStats.bucketDelayedIndexStats = - ((PersistentDispatcherMultipleConsumers) dispatcher).getBucketDelayedIndexStats(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getBucketDelayedIndexStats(); } if (Subscription.isIndividualAckMode(subType)) { - if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers d = (PersistentDispatcherMultipleConsumers) dispatcher; + if (dispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers d = + (AbstractPersistentDispatcherMultipleConsumers) dispatcher; subStats.unackedMessages = d.getTotalUnackedMessages(); subStats.blockedSubscriptionOnUnackedMsgs = d.isBlockedDispatcherOnUnackedMsgs(); subStats.msgDelayed = d.getNumberOfDelayedMessages(); @@ -1290,12 +1306,18 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge subStats.isReplicated = isReplicated(); subStats.subscriptionProperties = subscriptionProperties; subStats.isDurable = cursor.isDurable(); - if (getType() == SubType.Key_Shared && dispatcher instanceof PersistentStickyKeyDispatcherMultipleConsumers) { - PersistentStickyKeyDispatcherMultipleConsumers keySharedDispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) dispatcher; - + if (getType() == SubType.Key_Shared && dispatcher instanceof StickyKeyDispatcher) { + StickyKeyDispatcher keySharedDispatcher = (StickyKeyDispatcher) dispatcher; subStats.allowOutOfOrderDelivery = keySharedDispatcher.isAllowOutOfOrderDelivery(); subStats.keySharedMode = keySharedDispatcher.getKeySharedMode().toString(); + + LinkedHashMap recentlyJoinedConsumers = keySharedDispatcher + .getRecentlyJoinedConsumers(); + if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) { + recentlyJoinedConsumers.forEach((k, v) -> { + subStats.consumersAfterMarkDeletePosition.put(k.consumerName(), v.toString()); + }); + } } subStats.nonContiguousDeletedMessagesRanges = cursor.getTotalNonContiguousDeletedMessagesRange(); subStats.nonContiguousDeletedMessagesRangesSerializedSize = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 3cce175660e70..9c86a99de0f14 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2449,9 +2449,9 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats } if (Subscription.isIndividualAckMode(subscription.getType())) { - if (subscription.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) subscription.getDispatcher(); + if (subscription.getDispatcher() instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) subscription.getDispatcher(); topicStatsStream.writePair("blockedSubscriptionOnUnackedMsgs", dispatcher.isBlockedDispatcherOnUnackedMsgs()); topicStatsStream.writePair("unackedMessages", @@ -2758,11 +2758,11 @@ public CompletableFuture getInternalStats(boolean // subscription metrics PersistentSubscription sub = subscriptions.get(Codec.decode(c.getName())); if (sub != null) { - if (sub.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) sub.getDispatcher(); - cs.subscriptionHavePendingRead = dispatcher.havePendingRead; - cs.subscriptionHavePendingReplayRead = dispatcher.havePendingReplayRead; + if (sub.getDispatcher() instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) sub.getDispatcher(); + cs.subscriptionHavePendingRead = dispatcher.isHavePendingRead(); + cs.subscriptionHavePendingReplayRead = dispatcher.isHavePendingReplayRead(); } else if (sub.getDispatcher() instanceof PersistentDispatcherSingleActiveConsumer) { PersistentDispatcherSingleActiveConsumer dispatcher = (PersistentDispatcherSingleActiveConsumer) sub.getDispatcher(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index ffcc3bf0881db..7ed4542b2505f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -36,6 +36,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.BiFunction; +import java.util.stream.Stream; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -207,14 +208,28 @@ public static void receiveMessagesN(BiConsumer, Message> mess * Receive messages concurrently from multiple consumers and handles them using the provided message handler. * * @param messageHandler the message handler - * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages - * @param consumers the consumers to receive messages from - * @param the message value type + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param consumers the consumers to receive messages from + * @param the message value type + */ + public static void receiveMessagesInThreads(BiFunction, Message, Boolean> messageHandler, + final Duration quietTimeout, + Consumer... consumers) { + receiveMessagesInThreads(messageHandler, quietTimeout, Arrays.stream(consumers).sequential()); + } + + /** + * Receive messages concurrently from multiple consumers and handles them using the provided message handler. + * + * @param messageHandler the message handler + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param consumers the consumers to receive messages from + * @param the message value type */ public static void receiveMessagesInThreads(BiFunction, Message, Boolean> messageHandler, final Duration quietTimeout, - Consumer... consumers) { - FutureUtil.waitForAll(Arrays.stream(consumers).sequential().map(consumer -> { + Stream> consumers) { + FutureUtil.waitForAll(consumers.map(consumer -> { return CompletableFuture.runAsync(() -> { try { while (!Thread.currentThread().isInterrupted()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java index ea6ffa2d70dba..cdd0be58b34d5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java @@ -38,7 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.mledger.Position; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; @@ -49,7 +49,7 @@ public abstract class AbstractDeliveryTrackerTest { protected final Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-in-memory-delayed-delivery-test"), 500, TimeUnit.MILLISECONDS); - protected PersistentDispatcherMultipleConsumers dispatcher; + protected AbstractPersistentDispatcherMultipleConsumers dispatcher; protected Clock clock; protected AtomicLong clockTime; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java index 9861ab5723732..ff6bf534129c1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java @@ -24,6 +24,7 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -67,10 +68,10 @@ public void cleanup() throws Exception { @Test public void testFallbackToInMemoryTracker() throws Exception { - Pair pair = + Pair pair = mockDelayedDeliveryTrackerFactoryAndDispatcher(); BrokerService brokerService = pair.getLeft(); - PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); // Since Mocked BucketDelayedDeliveryTrackerFactory.newTracker0() throws RecoverDelayedDeliveryTrackerException, // the factory should be fallback to InMemoryDelayedDeliveryTrackerFactory @@ -83,12 +84,13 @@ public void testFallbackToInMemoryTracker() throws Exception { } - private Pair mockDelayedDeliveryTrackerFactoryAndDispatcher() + private Pair mockDelayedDeliveryTrackerFactoryAndDispatcher() throws Exception { BrokerService brokerService = Mockito.spy(pulsar.getBrokerService()); // Mock dispatcher - PersistentDispatcherMultipleConsumers dispatcher = Mockito.mock(PersistentDispatcherMultipleConsumers.class); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + Mockito.mock(AbstractPersistentDispatcherMultipleConsumers.class); Mockito.doReturn("test").when(dispatcher).getName(); // Mock BucketDelayedDeliveryTrackerFactory @Cleanup @@ -113,10 +115,10 @@ private Pair mockDelayedDe @Test public void testFallbackToInMemoryTrackerFactoryFailed() throws Exception { - Pair pair = + Pair pair = mockDelayedDeliveryTrackerFactoryAndDispatcher(); BrokerService brokerService = pair.getLeft(); - PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); // Mock InMemoryDelayedDeliveryTrackerFactory @Cleanup diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java index 6711aed924c20..ff7763927d888 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java @@ -37,7 +37,7 @@ import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -46,7 +46,7 @@ public class InMemoryDeliveryTrackerTest extends AbstractDeliveryTrackerTest { @DataProvider(name = "delayedTracker") public Object[][] provider(Method method) throws Exception { - dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + dispatcher = mock(AbstractPersistentDispatcherMultipleConsumers.class); clock = mock(Clock.class); clockTime = new AtomicLong(); when(clock.millis()).then(x -> clockTime.get()); @@ -212,7 +212,8 @@ public void testClose() throws Exception { Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-in-memory-delayed-delivery-test"), 1, TimeUnit.MILLISECONDS); - PersistentDispatcherMultipleConsumers dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + mock(AbstractPersistentDispatcherMultipleConsumers.class); AtomicLong clockTime = new AtomicLong(); Clock clock = mock(Clock.class); 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 bf5a282a4ee6d..426bd50c96bbb 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 @@ -51,7 +51,7 @@ import org.apache.pulsar.broker.delayed.AbstractDeliveryTrackerTest; import org.apache.pulsar.broker.delayed.MockBucketSnapshotStorage; import org.apache.pulsar.broker.delayed.MockManagedCursor; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.awaitility.Awaitility; import org.roaringbitmap.RoaringBitmap; import org.roaringbitmap.buffer.ImmutableRoaringBitmap; @@ -74,7 +74,7 @@ public void clean() throws Exception { @DataProvider(name = "delayedTracker") public Object[][] provider(Method method) throws Exception { - dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + dispatcher = mock(AbstractPersistentDispatcherMultipleConsumers.class); clock = mock(Clock.class); clockTime = new AtomicLong(); when(clock.millis()).then(x -> clockTime.get()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java index c66eff2c8a180..2fd288239e362 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java @@ -37,7 +37,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.BatcherBuilder; @@ -786,7 +786,7 @@ public void testConcurrentBatchMessageAck(BatcherBuilder builder) throws Excepti } latch.await(); - PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) topic + AbstractPersistentDispatcherMultipleConsumers dispatcher = (AbstractPersistentDispatcherMultipleConsumers) topic .getSubscription(subscriptionName).getDispatcher(); // check strategically to let ack-message receive by broker retryStrategically((test) -> dispatcher.getConsumers().get(0).getUnackedMessages() == 0, 50, 150); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java index ed7f6974dd26f..21a843a3efc22 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java @@ -41,7 +41,7 @@ import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.BrokerTestUtil; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -106,7 +106,7 @@ public void testBatchMessageAck() { } FutureUtil.waitForAll(sendFutureList).get(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); - PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) topic + AbstractPersistentDispatcherMultipleConsumers dispatcher = (AbstractPersistentDispatcherMultipleConsumers) topic .getSubscription(subscriptionName).getDispatcher(); Message receive1 = consumer.receive(); Message receive2 = consumer.receive(); @@ -515,8 +515,8 @@ private BitSetRecyclable createBitSetRecyclable(int batchSize) { private ManagedCursorImpl getCursor(String topic, String sub) { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); return (ManagedCursorImpl) dispatcher.getCursor(); } @@ -528,8 +528,8 @@ private org.apache.pulsar.broker.service.Consumer makeConsumerReceiveMessagesDel CompletableFuture signal) throws Exception { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); org.apache.pulsar.broker.service.Consumer serviceConsumer = null; for (org.apache.pulsar.broker.service.Consumer c : dispatcher.getConsumers()){ if (c.consumerName().equals(consumerName)) { @@ -664,8 +664,8 @@ public void testPermitsIfHalfAckBatchMessage() throws Exception { private org.apache.pulsar.broker.service.Consumer getTheUniqueServiceConsumer(String topic, String sub) { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService(). getTopic(topic, false).join().get(); - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); return dispatcher.getConsumers().iterator().next(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java index 69f3e2e4d3917..2899e9f2d67db 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java @@ -66,6 +66,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -526,7 +527,8 @@ public void testMultipleDispatcherGetNextConsumerWithDifferentPriorityLevel() th PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, false, 1); Consumer consumer2 = createConsumer(topic, 0, 2, false, 2); Consumer consumer3 = createConsumer(topic, 0, 2, false, 3); @@ -571,7 +573,8 @@ public void testMultipleDispatcherGetNextConsumerWithDifferentPriorityLevel() th public void testFewBlockedConsumerSamePriority() throws Exception{ PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, false, 1); Consumer consumer2 = createConsumer(topic, 0, 2, false, 2); Consumer consumer3 = createConsumer(topic, 0, 2, false, 3); @@ -599,7 +602,8 @@ public void testFewBlockedConsumerSamePriority() throws Exception{ public void testFewBlockedConsumerDifferentPriority() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, false, 1); Consumer consumer2 = createConsumer(topic, 0, 2, false, 2); Consumer consumer3 = createConsumer(topic, 0, 2, false, 3); @@ -654,7 +658,8 @@ public void testFewBlockedConsumerDifferentPriority() throws Exception { public void testFewBlockedConsumerDifferentPriority2() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, true, 1); Consumer consumer2 = createConsumer(topic, 0, 2, true, 2); Consumer consumer3 = createConsumer(topic, 0, 2, true, 3); @@ -677,7 +682,7 @@ public void testFewBlockedConsumerDifferentPriority2() throws Exception { } @SuppressWarnings("unchecked") - private Consumer getNextConsumer(PersistentDispatcherMultipleConsumers dispatcher) throws Exception { + private Consumer getNextConsumer(AbstractPersistentDispatcherMultipleConsumers dispatcher) throws Exception { Consumer consumer = dispatcher.getNextConsumer(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index 1e96da737dd51..92b767104f6cf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -99,6 +99,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; @@ -346,7 +347,7 @@ public void testDispatcherMultiConsumerReadFailed() { when(cursor.getName()).thenReturn("cursor"); Subscription subscription = mock(Subscription.class); when(subscription.getName()).thenReturn("sub"); - PersistentDispatcherMultipleConsumers dispatcher = + AbstractPersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, subscription); dispatcher.readEntriesFailed(new ManagedLedgerException.InvalidCursorPositionException("failed"), null); verify(topic, atLeast(1)).getBrokerService(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 20ea33fb3e1ed..4c8e6897df3fc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -102,8 +102,8 @@ public void testBucketDelayedDeliveryWithAllConsumersDisconnecting() throws Exce Dispatcher dispatcher = pulsar.getBrokerService().getTopicReference(topic).get().getSubscription("sub").getDispatcher(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); List bucketKeys = - ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties().keySet().stream() - .filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties().keySet() + .stream().filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); c1.close(); @@ -117,8 +117,8 @@ public void testBucketDelayedDeliveryWithAllConsumersDisconnecting() throws Exce Dispatcher dispatcher2 = pulsar.getBrokerService().getTopicReference(topic).get().getSubscription("sub").getDispatcher(); List bucketKeys2 = - ((PersistentDispatcherMultipleConsumers) dispatcher2).getCursor().getCursorProperties().keySet().stream() - .filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher2).getCursor().getCursorProperties().keySet() + .stream().filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher2.getNumberOfDelayedMessages(), 1000)); Assert.assertEquals(bucketKeys, bucketKeys2); @@ -152,7 +152,7 @@ public void testUnsubscribe() throws Exception { Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); Map cursorProperties = - ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); List bucketIds = cursorProperties.entrySet().stream() .filter(x -> x.getKey().startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX + "delayed.bucket")).map( x -> Long.valueOf(x.getValue())).toList(); @@ -339,7 +339,7 @@ public void testDelete() throws Exception { Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); Map cursorProperties = - ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); List bucketIds = cursorProperties.entrySet().stream() .filter(x -> x.getKey().startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX + "delayed.bucket")).map( x -> Long.valueOf(x.getValue())).toList(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index 3ca966d210886..e47857e8ec60f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -259,12 +259,14 @@ public void testDelayedDeliveryWithMultipleConcurrentReadEntries() .subscribe(); // Simulate race condition with high frequency of calls to dispatcher.readMoreEntries() - PersistentDispatcherMultipleConsumers d = (PersistentDispatcherMultipleConsumers) ((PersistentTopic) pulsar - .getBrokerService().getTopicReference(topic).get()).getSubscription("shared-sub").getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers d = + (AbstractPersistentDispatcherMultipleConsumers) ((PersistentTopic) pulsar + .getBrokerService().getTopicReference(topic).get()).getSubscription("shared-sub") + .getDispatcher(); Thread t = new Thread(() -> { while (true) { synchronized (d) { - d.readMoreEntries(); + d.readMoreEntriesAsync(); } try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java new file mode 100644 index 0000000000000..487d99891fd3a --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java @@ -0,0 +1,172 @@ +/* + * 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.service.persistent; + +import com.carrotsearch.hppc.ObjectSet; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.client.api.Consumer; +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.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.awaitility.reflect.WhiteboxImpl; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class PersistentDispatcherMultipleConsumersClassicTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + List consumerList = + WhiteboxImpl.getInternalState(dispatcher, "consumerList"); + + org.apache.pulsar.broker.service.Consumer serviceConsumer = consumerList.get(0); + consumerSet.add(serviceConsumer); + consumerList.add(serviceConsumer); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList2() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + consumerSet.clear(); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } + + @Test + public void testSkipReadEntriesFromCloseCursor() throws Exception { + final String topicName = + BrokerTestUtil.newUniqueName("persistent://public/default/testSkipReadEntriesFromCloseCursor"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + for (int i = 0; i < 10; i++) { + producer.send("message-" + i); + } + producer.close(); + + // Get the dispatcher of the topic. + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService() + .getTopic(topicName, false).join().get(); + + ManagedCursor cursor = Mockito.mock(ManagedCursorImpl.class); + Mockito.doReturn(subscription).when(cursor).getName(); + Subscription sub = Mockito.mock(PersistentSubscription.class); + Mockito.doReturn(topic).when(sub).getTopic(); + // Mock the dispatcher. + PersistentDispatcherMultipleConsumersClassic dispatcher = + Mockito.spy(new PersistentDispatcherMultipleConsumersClassic(topic, cursor, sub)); + // Return 10 permits to make the dispatcher can read more entries. + Mockito.doReturn(10).when(dispatcher).getFirstAvailableConsumerPermits(); + + // Make the count + 1 when call the scheduleReadEntriesWithDelay(...). + AtomicInteger callScheduleReadEntriesWithDelayCnt = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callScheduleReadEntriesWithDelayCnt.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).scheduleReadEntriesWithDelay(Mockito.any(), Mockito.any(), Mockito.anyLong()); + + // Make the count + 1 when call the readEntriesFailed(...). + AtomicInteger callReadEntriesFailed = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callReadEntriesFailed.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).readEntriesFailed(Mockito.any(), Mockito.any()); + + Mockito.doReturn(false).when(cursor).isClosed(); + + // Mock the readEntriesOrWait(...) to simulate the cursor is closed. + Mockito.doAnswer(inv -> { + PersistentDispatcherMultipleConsumersClassic dispatcher1 = inv.getArgument(2); + dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), + null); + return null; + }).when(cursor).asyncReadEntriesOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher), + Mockito.any(), Mockito.any()); + + dispatcher.readMoreEntries(); + + // Verify: the readEntriesFailed should be called once and the scheduleReadEntriesWithDelay should not be called. + Assert.assertTrue(callReadEntriesFailed.get() == 1 && callScheduleReadEntriesWithDelayCnt.get() == 0); + + // Verify: the topic can be deleted successfully. + admin.topics().delete(topicName, false); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java index 052c5ceb5cdde..772b1843d2894 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java @@ -154,7 +154,7 @@ public void testSkipReadEntriesFromCloseCursor() throws Exception { // Mock the readEntriesOrWait(...) to simulate the cursor is closed. Mockito.doAnswer(inv -> { - PersistentDispatcherMultipleConsumers dispatcher1 = inv.getArgument(2); + AbstractPersistentDispatcherMultipleConsumers dispatcher1 = inv.getArgument(2); dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), null); return null; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java new file mode 100644 index 0000000000000..1f40fd46aa344 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java @@ -0,0 +1,482 @@ +/* + * 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.service.persistent; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.common.protocol.Commands.serializeMetadataAndPayload; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anySet; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelPromise; +import io.netty.channel.EventLoopGroup; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.EntryImpl; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; +import org.apache.pulsar.broker.service.EntryBatchSizes; +import org.apache.pulsar.broker.service.RedeliveryTracker; +import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; +import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.api.proto.KeySharedMode; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.policies.data.HierarchyTopicPolicies; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.protocol.Markers; +import org.awaitility.Awaitility; +import org.mockito.ArgumentCaptor; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class PersistentStickyKeyDispatcherMultipleConsumersClassicTest { + + private PulsarService pulsarMock; + private BrokerService brokerMock; + private ManagedCursorImpl cursorMock; + private Consumer consumerMock; + private PersistentTopic topicMock; + private PersistentSubscription subscriptionMock; + private ServiceConfiguration configMock; + private ChannelPromise channelMock; + private OrderedExecutor orderedExecutor; + + private PersistentStickyKeyDispatcherMultipleConsumersClassic persistentDispatcher; + + final String topicName = "persistent://public/default/testTopic"; + final String subscriptionName = "testSubscription"; + + @BeforeMethod + public void setup() throws Exception { + configMock = mock(ServiceConfiguration.class); + doReturn(true).when(configMock).isSubscriptionRedeliveryTrackerEnabled(); + doReturn(100).when(configMock).getDispatcherMaxReadBatchSize(); + doReturn(true).when(configMock).isSubscriptionKeySharedUseConsistentHashing(); + doReturn(1).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); + doReturn(true).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); + doReturn(false).when(configMock).isAllowOverrideEntryFilters(); + + pulsarMock = mock(PulsarService.class); + doReturn(configMock).when(pulsarMock).getConfiguration(); + + EntryFilterProvider mockEntryFilterProvider = mock(EntryFilterProvider.class); + when(mockEntryFilterProvider.getBrokerEntryFilters()).thenReturn(Collections.emptyList()); + + brokerMock = mock(BrokerService.class); + doReturn(pulsarMock).when(brokerMock).pulsar(); + when(brokerMock.getEntryFilterProvider()).thenReturn(mockEntryFilterProvider); + + HierarchyTopicPolicies topicPolicies = new HierarchyTopicPolicies(); + topicPolicies.getMaxConsumersPerSubscription().updateBrokerValue(0); + + orderedExecutor = OrderedExecutor.newBuilder().build(); + doReturn(orderedExecutor).when(brokerMock).getTopicOrderedExecutor(); + + EventLoopGroup eventLoopGroup = mock(EventLoopGroup.class); + doReturn(eventLoopGroup).when(brokerMock).executor(); + doAnswer(invocation -> { + orderedExecutor.execute(((Runnable)invocation.getArguments()[0])); + return null; + }).when(eventLoopGroup).execute(any(Runnable.class)); + + topicMock = mock(PersistentTopic.class); + doReturn(brokerMock).when(topicMock).getBrokerService(); + doReturn(topicName).when(topicMock).getName(); + doReturn(topicPolicies).when(topicMock).getHierarchyTopicPolicies(); + + cursorMock = mock(ManagedCursorImpl.class); + doReturn(null).when(cursorMock).getLastIndividualDeletedRange(); + doReturn(subscriptionName).when(cursorMock).getName(); + + consumerMock = mock(Consumer.class); + channelMock = mock(ChannelPromise.class); + doReturn("consumer1").when(consumerMock).consumerName(); + doReturn(1000).when(consumerMock).getAvailablePermits(); + doReturn(true).when(consumerMock).isWritable(); + doReturn(channelMock).when(consumerMock).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + + subscriptionMock = mock(PersistentSubscription.class); + when(subscriptionMock.getTopic()).thenReturn(topicMock); + persistentDispatcher = new PersistentStickyKeyDispatcherMultipleConsumersClassic( + topicMock, cursorMock, subscriptionMock, configMock, + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)); + } + + @AfterMethod(alwaysRun = true) + public void cleanup() { + if (persistentDispatcher != null && !persistentDispatcher.isClosed()) { + persistentDispatcher.close(); + } + if (orderedExecutor != null) { + orderedExecutor.shutdownNow(); + orderedExecutor = null; + } + } + + @Test(timeOut = 10000) + public void testAddConsumerWhenClosed() throws Exception { + persistentDispatcher.close().get(); + Consumer consumer = mock(Consumer.class); + persistentDispatcher.addConsumer(consumer); + verify(consumer, times(1)).disconnect(); + assertEquals(0, persistentDispatcher.getConsumers().size()); + assertTrue(persistentDispatcher.getSelector().getConsumerKeyHashRanges().isEmpty()); + } + + @Test + public void testSendMarkerMessage() { + try { + persistentDispatcher.addConsumer(consumerMock); + persistentDispatcher.consumerFlow(consumerMock, 1000); + } catch (Exception e) { + fail("Failed to add mock consumer", e); + } + + List entries = new ArrayList<>(); + ByteBuf markerMessage = Markers.newReplicatedSubscriptionsSnapshotRequest("testSnapshotId", "testSourceCluster"); + entries.add(EntryImpl.create(1, 1, markerMessage)); + entries.add(EntryImpl.create(1, 2, createMessage("message1", 1))); + entries.add(EntryImpl.create(1, 3, createMessage("message2", 2))); + entries.add(EntryImpl.create(1, 4, createMessage("message3", 3))); + entries.add(EntryImpl.create(1, 5, createMessage("message4", 4))); + entries.add(EntryImpl.create(1, 6, createMessage("message5", 5))); + + try { + persistentDispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } catch (Exception e) { + fail("Failed to readEntriesComplete.", e); + } + + Awaitility.await().untilAsserted(() -> { + ArgumentCaptor totalMessagesCaptor = ArgumentCaptor.forClass(Integer.class); + verify(consumerMock, times(1)).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + totalMessagesCaptor.capture(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + + List allTotalMessagesCaptor = totalMessagesCaptor.getAllValues(); + Assert.assertEquals(allTotalMessagesCaptor.get(0).intValue(), 5); + }); + } + + @Test(timeOut = 10000) + public void testSendMessage() { + KeySharedMeta keySharedMeta = new KeySharedMeta().setKeySharedMode(KeySharedMode.STICKY); + PersistentStickyKeyDispatcherMultipleConsumersClassic + persistentDispatcher = new PersistentStickyKeyDispatcherMultipleConsumersClassic( + topicMock, cursorMock, subscriptionMock, configMock, keySharedMeta); + try { + keySharedMeta.addHashRange() + .setStart(0) + .setEnd(9); + + Consumer consumerMock = mock(Consumer.class); + doReturn(keySharedMeta).when(consumerMock).getKeySharedMeta(); + persistentDispatcher.addConsumer(consumerMock); + persistentDispatcher.consumerFlow(consumerMock, 1000); + } catch (Exception e) { + fail("Failed to add mock consumer", e); + } + + List entries = new ArrayList<>(); + entries.add(EntryImpl.create(1, 1, createMessage("message1", 1))); + entries.add(EntryImpl.create(1, 2, createMessage("message2", 2))); + + try { + //Should success,see issue #8960 + persistentDispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } catch (Exception e) { + fail("Failed to readEntriesComplete.", e); + } + } + + @Test + public void testSkipRedeliverTemporally() { + final Consumer slowConsumerMock = mock(Consumer.class); + final ChannelPromise slowChannelMock = mock(ChannelPromise.class); + // add entries to redeliver and read target + final List redeliverEntries = new ArrayList<>(); + redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key1"))); + final List readEntries = new ArrayList<>(); + readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); + readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key2"))); + + try { + Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumersClassic.class.getDeclaredField("totalAvailablePermits"); + totalAvailablePermitsField.setAccessible(true); + totalAvailablePermitsField.set(persistentDispatcher, 1000); + + doAnswer(invocationOnMock -> { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(2)) + .readEntriesComplete(readEntries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + return null; + }).when(cursorMock).asyncReadEntriesOrWait( + anyInt(), anyLong(), any(PersistentStickyKeyDispatcherMultipleConsumersClassic.class), + eq(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal), any()); + } catch (Exception e) { + fail("Failed to set to field", e); + } + + // Create 2Consumers + try { + doReturn("consumer2").when(slowConsumerMock).consumerName(); + // Change slowConsumer availablePermits to 0 and back to normal + when(slowConsumerMock.getAvailablePermits()) + .thenReturn(0) + .thenReturn(1); + doReturn(true).when(slowConsumerMock).isWritable(); + doReturn(slowChannelMock).when(slowConsumerMock).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + + persistentDispatcher.addConsumer(consumerMock); + persistentDispatcher.addConsumer(slowConsumerMock); + } catch (Exception e) { + fail("Failed to add mock consumer", e); + } + + // run PersistentStickyKeyDispatcherMultipleConsumers#sendMessagesToConsumers + // run readMoreEntries internally (and skip internally) + // Change slowConsumer availablePermits to 1 + // run PersistentStickyKeyDispatcherMultipleConsumers#sendMessagesToConsumers internally + // and then stop to dispatch to slowConsumer + if (persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal, + redeliverEntries, true)) { + persistentDispatcher.readMoreEntriesAsync(); + } + + Awaitility.await().untilAsserted(() -> { + verify(consumerMock, times(1)).sendMessages( + argThat(arg -> { + assertEquals(arg.size(), 1); + Entry entry = arg.get(0); + assertEquals(entry.getLedgerId(), 1); + assertEquals(entry.getEntryId(), 3); + return true; + }), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + }); + verify(slowConsumerMock, times(0)).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + } + + @Test(timeOut = 30000) + public void testMessageRedelivery() throws Exception { + final Queue actualEntriesToConsumer1 = new ConcurrentLinkedQueue<>(); + final Queue actualEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); + + final Queue expectedEntriesToConsumer1 = new ConcurrentLinkedQueue<>(); + expectedEntriesToConsumer1.add(PositionFactory.create(1, 1)); + final Queue expectedEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); + expectedEntriesToConsumer2.add(PositionFactory.create(1, 2)); + expectedEntriesToConsumer2.add(PositionFactory.create(1, 3)); + + final AtomicInteger remainingEntriesNum = new AtomicInteger( + expectedEntriesToConsumer1.size() + expectedEntriesToConsumer2.size()); + + // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2 + final List allEntries = new ArrayList<>(); + allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key2"))); + allEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); + allEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key1"))); + allEntries.forEach(entry -> ((EntryImpl) entry).retain()); + + final List redeliverEntries = new ArrayList<>(); + redeliverEntries.add(allEntries.get(0)); // message1 + final List readEntries = new ArrayList<>(); + readEntries.add(allEntries.get(2)); // message3 + + final Consumer consumer1 = mock(Consumer.class); + doReturn("consumer1").when(consumer1).consumerName(); + // Change availablePermits of consumer1 to 0 and then back to normal + when(consumer1.getAvailablePermits()).thenReturn(0).thenReturn(10); + doReturn(true).when(consumer1).isWritable(); + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + List entries = (List) invocationOnMock.getArgument(0); + for (Entry entry : entries) { + remainingEntriesNum.decrementAndGet(); + actualEntriesToConsumer1.add(entry.getPosition()); + } + return channelMock; + }).when(consumer1).sendMessages(anyList(), any(EntryBatchSizes.class), any(EntryBatchIndexesAcks.class), + anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + + final Consumer consumer2 = mock(Consumer.class); + doReturn("consumer2").when(consumer2).consumerName(); + when(consumer2.getAvailablePermits()).thenReturn(10); + doReturn(true).when(consumer2).isWritable(); + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + List entries = (List) invocationOnMock.getArgument(0); + for (Entry entry : entries) { + remainingEntriesNum.decrementAndGet(); + actualEntriesToConsumer2.add(entry.getPosition()); + } + return channelMock; + }).when(consumer2).sendMessages(anyList(), any(EntryBatchSizes.class), any(EntryBatchIndexesAcks.class), + anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + + persistentDispatcher.addConsumer(consumer1); + persistentDispatcher.addConsumer(consumer2); + + final Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumersClassic.class + .getDeclaredField("totalAvailablePermits"); + totalAvailablePermitsField.setAccessible(true); + totalAvailablePermitsField.set(persistentDispatcher, 1000); + + final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumersClassic.class + .getDeclaredField("redeliveryMessages"); + redeliveryMessagesField.setAccessible(true); + MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField + .get(persistentDispatcher); + redeliveryMessages.add(allEntries.get(0).getLedgerId(), allEntries.get(0).getEntryId(), + persistentDispatcher.getStickyKeyHash(allEntries.get(0))); // message1 + redeliveryMessages.add(allEntries.get(1).getLedgerId(), allEntries.get(1).getEntryId(), + persistentDispatcher.getStickyKeyHash(allEntries.get(1))); // message2 + + // Mock Cursor#asyncReplayEntries + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + Set positions = (Set) invocationOnMock.getArgument(0); + List entries = allEntries.stream().filter(entry -> positions.contains(entry.getPosition())) + .collect(Collectors.toList()); + if (!entries.isEmpty()) { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(1)) + .readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Replay); + } + return Collections.emptySet(); + }).when(cursorMock).asyncReplayEntries(anySet(), any(PersistentStickyKeyDispatcherMultipleConsumersClassic.class), + eq(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Replay), anyBoolean()); + + // Mock Cursor#asyncReadEntriesOrWait + AtomicBoolean asyncReadEntriesOrWaitCalled = new AtomicBoolean(); + doAnswer(invocationOnMock -> { + if (asyncReadEntriesOrWaitCalled.compareAndSet(false, true)) { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(2)) + .readEntriesComplete(readEntries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } else { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(2)) + .readEntriesComplete(Collections.emptyList(), PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } + return null; + }).when(cursorMock).asyncReadEntriesOrWait(anyInt(), anyLong(), + any(PersistentStickyKeyDispatcherMultipleConsumersClassic.class), + eq(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal), any()); + + // (1) Run sendMessagesToConsumers + // (2) Attempts to send message1 to consumer1 but skipped because availablePermits is 0 + // (3) Change availablePermits of consumer1 to 10 + // (4) Run readMoreEntries internally + // (5) Run sendMessagesToConsumers internally + // (6) Attempts to send message3 to consumer2 but skipped because redeliveryMessages contains message2 + persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Replay, + redeliverEntries, true); + while (remainingEntriesNum.get() > 0) { + // (7) Run readMoreEntries and resend message1 to consumer1 and message2-3 to consumer2 + persistentDispatcher.readMoreEntries(); + } + + assertThat(actualEntriesToConsumer1).containsExactlyElementsOf(expectedEntriesToConsumer1); + assertThat(actualEntriesToConsumer2).containsExactlyElementsOf(expectedEntriesToConsumer2); + + allEntries.forEach(entry -> entry.release()); + } + + private ByteBuf createMessage(String message, int sequenceId) { + return createMessage(message, sequenceId, "testKey"); + } + + private ByteBuf createMessage(String message, int sequenceId, String key) { + MessageMetadata messageMetadata = new MessageMetadata() + .setSequenceId(sequenceId) + .setProducerName("testProducer") + .setPartitionKey(key) + .setPartitionKeyB64Encoded(false) + .setPublishTime(System.currentTimeMillis()); + return serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, Unpooled.copiedBuffer(message.getBytes(UTF_8))); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index 4b29ead984e7a..7234f0caefc63 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -720,7 +720,7 @@ public void testNoBackoffDelayWhenDelayedMessages(boolean dispatchMessagesInSubs AtomicInteger reScheduleReadInMsCalled = new AtomicInteger(0); AtomicBoolean delayAllMessages = new AtomicBoolean(true); - PersistentDispatcherMultipleConsumers dispatcher; + AbstractPersistentDispatcherMultipleConsumers dispatcher; if (isKeyShared) { dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers( topicMock, cursorMock, subscriptionMock, configMock, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index 14403765105b9..5b2998216e8e1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -233,7 +233,7 @@ public void testConsumerStatsOutput() throws Exception { "unackedMessages", "avgMessagesPerEntry", "blockedConsumerOnUnackedMsgs", - "lastSentPositionWhenJoining", + "readPositionWhenJoining", "lastAckedTime", "lastAckedTimestamp", "lastConsumedTime", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 2b16647f5590c..08efb6d9583ef 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -65,10 +65,13 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.DrainingHashesTracker; import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; @@ -79,6 +82,7 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.tests.KeySharedImplementationType; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.mockito.Mockito; @@ -89,53 +93,80 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker-impl") public class KeySharedSubscriptionTest extends ProducerConsumerBase { - private static final Logger log = LoggerFactory.getLogger(KeySharedSubscriptionTest.class); private static final List keys = Arrays.asList("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"); private static final String SUBSCRIPTION_NAME = "key_shared"; + private final KeySharedImplementationType implementationType; + + // Comment out the next line (Factory annotation) to run tests manually in IntelliJ, one-by-one + @Factory + public static Object[] createTestInstances() { + return KeySharedImplementationType.generateTestInstances(KeySharedSubscriptionTest::new); + } + + public KeySharedSubscriptionTest() { + // set the default implementation type for manual running in IntelliJ + this(KeySharedImplementationType.DEFAULT); + } + + public KeySharedSubscriptionTest(KeySharedImplementationType implementationType) { + this.implementationType = implementationType; + } + + private Object[][] prependImplementationTypeToData(Object[][] data) { + return implementationType.prependImplementationTypeToData(data); + } + + @DataProvider(name = "currentImplementationType") + public Object[] currentImplementationType() { + return new Object[]{ implementationType }; + } @DataProvider(name = "batch") - public Object[] batchProvider() { - return new Object[] { - false, - true - }; + public Object[][] batchProvider() { + return prependImplementationTypeToData(new Object[][]{ + {false}, + {true} + }); } @DataProvider(name = "partitioned") public Object[][] partitionedProvider() { - return new Object[][] { - { false }, - { true } - }; + return prependImplementationTypeToData(new Object[][]{ + {false}, + {true} + }); } @DataProvider(name = "data") public Object[][] dataProvider() { - return new Object[][] { + return prependImplementationTypeToData(new Object[][]{ // Topic-Type and "Batching" - { "persistent", false }, - { "persistent", true }, - { "non-persistent", false }, - { "non-persistent", true }, - }; + {"persistent", false}, + {"persistent", true}, + {"non-persistent", false}, + {"non-persistent", true}, + }); } @DataProvider(name = "topicDomain") public Object[][] topicDomainProvider() { - return new Object[][] { - { "persistent" }, - { "non-persistent" } - }; + return prependImplementationTypeToData(new Object[][]{ + {"persistent"}, + {"non-persistent"} + }); } @BeforeClass(alwaysRun = true) @Override protected void setup() throws Exception { + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(implementationType.classic); + conf.setSubscriptionSharedUseClassicPersistentImplementation(implementationType.classic); this.conf.setUnblockStuckSubscriptionEnabled(true); super.internalSetup(); super.producerBaseSetup(); @@ -170,7 +201,8 @@ public void resetDefaultNamespace() throws Exception { private static final int NUMBER_OF_KEYS = 300; @Test(dataProvider = "data") - public void testSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector(String topicType, boolean enableBatch) + public void testSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector(KeySharedImplementationType impl, + String topicType, boolean enableBatch) throws PulsarClientException { String topic = topicType + "://public/default/key_shared-" + UUID.randomUUID(); @@ -197,7 +229,7 @@ public void testSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector(St } @Test(dataProvider = "data") - public void testSendAndReceiveWithBatching(String topicType, boolean enableBatch) throws Exception { + public void testSendAndReceiveWithBatching(KeySharedImplementationType impl, String topicType, boolean enableBatch) throws Exception { String topic = topicType + "://public/default/key_shared-" + UUID.randomUUID(); @Cleanup @@ -242,7 +274,9 @@ public void testSendAndReceiveWithBatching(String topicType, boolean enableBatch } @Test(dataProvider = "batch") - public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) throws PulsarClientException { + public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) + throws PulsarClientException { String topic = "persistent://public/default/key_shared_exclusive-" + UUID.randomUUID(); @Cleanup @@ -294,8 +328,9 @@ public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(bo @Test(dataProvider = "data") public void testConsumerCrashSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( - String topicType, - boolean enableBatch + KeySharedImplementationType impl, + String topicType, + boolean enableBatch ) throws PulsarClientException, InterruptedException { String topic = topicType + "://public/default/key_shared_consumer_crash-" + UUID.randomUUID(); @@ -338,8 +373,9 @@ public void testConsumerCrashSendAndReceiveWithHashRangeAutoSplitStickyKeyConsum @Test(dataProvider = "data") public void testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( - String topicType, - boolean enableBatch + KeySharedImplementationType impl, + String topicType, + boolean enableBatch ) throws PulsarClientException { String topic = topicType + "://public/default/key_shared_no_key-" + UUID.randomUUID(); @@ -365,7 +401,8 @@ public void testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelect } @Test(dataProvider = "batch") - public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) + public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) throws PulsarClientException { String topic = "persistent://public/default/key_shared_no_key_exclusive-" + UUID.randomUUID(); @@ -415,7 +452,8 @@ public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelect } @Test(dataProvider = "batch") - public void testOrderingKeyWithHashRangeAutoSplitStickyKeyConsumerSelector(boolean enableBatch) + public void testOrderingKeyWithHashRangeAutoSplitStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) throws PulsarClientException { String topic = "persistent://public/default/key_shared_ordering_key-" + UUID.randomUUID(); @@ -443,7 +481,8 @@ public void testOrderingKeyWithHashRangeAutoSplitStickyKeyConsumerSelector(boole } @Test(dataProvider = "batch") - public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) + public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) throws PulsarClientException { String topic = "persistent://public/default/key_shared_exclusive_ordering_key-" + UUID.randomUUID(); @@ -512,8 +551,8 @@ public void testDisableKeySharedSubscription() throws PulsarClientException { } } - @Test - public void testCannotUseAcknowledgeCumulative() throws PulsarClientException { + @Test(dataProvider = "currentImplementationType") + public void testCannotUseAcknowledgeCumulative(KeySharedImplementationType impl) throws PulsarClientException { String topic = "persistent://public/default/key_shared_ack_cumulative-" + UUID.randomUUID(); @Cleanup @@ -538,7 +577,7 @@ public void testCannotUseAcknowledgeCumulative() throws PulsarClientException { } @Test(dataProvider = "batch") - public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exception { + public void testMakingProgressWithSlowerConsumer(KeySharedImplementationType impl, boolean enableBatch) throws Exception { String topic = "testMakingProgressWithSlowerConsumer-" + UUID.randomUUID(); String slowKey = "slowKey"; @@ -620,8 +659,8 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc } } - @Test - public void testOrderingWhenAddingConsumers() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testOrderingWhenAddingConsumers(KeySharedImplementationType impl) throws Exception { String topic = "testOrderingWhenAddingConsumers-" + UUID.randomUUID(); int numberOfKeys = 10; @@ -668,13 +707,13 @@ public void testOrderingWhenAddingConsumers() throws Exception { } @SneakyThrows - private PersistentStickyKeyDispatcherMultipleConsumers getDispatcher(String topic, String subscription) { - return (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get() + private StickyKeyDispatcher getDispatcher(String topic, String subscription) { + return (StickyKeyDispatcher) pulsar.getBrokerService().getTopicIfExists(topic).get() .get().getSubscription(subscription).getDispatcher(); } - @Test - public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testReadAheadWithConfiguredLookAheadLimit(KeySharedImplementationType impl) throws Exception { String topic = "testReadAheadWithConfiguredLookAheadLimit-" + UUID.randomUUID(); // Set the look ahead limit to 50 for subscriptions @@ -730,8 +769,8 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { assertTrue(entryId < 100); } - @Test - public void testRemoveFirstConsumer() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testRemoveFirstConsumer(KeySharedImplementationType impl) throws Exception { String topic = "testReadAheadWhenAddingConsumers-" + UUID.randomUUID(); int numberOfKeys = 10; @@ -788,8 +827,8 @@ public void testRemoveFirstConsumer() throws Exception { } } - @Test - public void testHashRangeConflict() throws PulsarClientException { + @Test(dataProvider = "currentImplementationType") + public void testHashRangeConflict(KeySharedImplementationType impl) throws PulsarClientException { final String topic = "persistent://public/default/testHashRangeConflict-" + UUID.randomUUID().toString(); final String sub = "test"; @@ -799,7 +838,7 @@ public void testHashRangeConflict() throws PulsarClientException { Consumer consumer2 = createFixedHashRangesConsumer(topic, sub, Range.of(100,399)); Assert.assertTrue(consumer2.isConnected()); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, sub); + StickyKeyDispatcher dispatcher = getDispatcher(topic, sub); Assert.assertEquals(dispatcher.getConsumers().size(), 2); try { @@ -849,8 +888,8 @@ public void testHashRangeConflict() throws PulsarClientException { Assert.assertFalse(dispatcher.isConsumerConnected()); } - @Test - public void testWithMessageCompression() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testWithMessageCompression(KeySharedImplementationType impl) throws Exception { final String topic = "testWithMessageCompression" + UUID.randomUUID().toString(); Producer producer = pulsarClient.newProducer() .topic(topic) @@ -876,8 +915,8 @@ public void testWithMessageCompression() throws Exception { consumer.close(); } - @Test - public void testAttachKeyToMessageMetadata() throws PulsarClientException { + @Test(dataProvider = "currentImplementationType") + public void testAttachKeyToMessageMetadata(KeySharedImplementationType impl) throws PulsarClientException { String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); @Cleanup @@ -904,8 +943,8 @@ public void testAttachKeyToMessageMetadata() throws PulsarClientException { receiveAndCheckDistribution(Lists.newArrayList(consumer1, consumer2, consumer3), 1000); } - @Test - public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testContinueDispatchMessagesWhenMessageTTL(KeySharedImplementationType impl) throws Exception { int defaultTTLSec = 3; int totalMessages = 1000; int numberOfKeys = 50; @@ -922,7 +961,7 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subName); + StickyKeyDispatcher dispatcher = getDispatcher(topic, subName); StickyKeyConsumerSelector selector = dispatcher.getSelector(); @Cleanup @@ -964,7 +1003,7 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { if (received != null) { int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); DrainingHashesTracker.DrainingHashEntry entry = - dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + !impl.classic ? getDrainingHashesTracker(dispatcher).getEntry(stickyKeyHash) : null; Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + "included in blockedHashes=%s", received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); @@ -982,10 +1021,10 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { received = consumer3.receive(1, TimeUnit.SECONDS); } catch (PulsarClientException ignore) { } - if (received != null) { + if (received != null && !impl.classic) { int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); DrainingHashesTracker.DrainingHashEntry entry = - dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + !impl.classic ? getDrainingHashesTracker(dispatcher).getEntry(stickyKeyHash) : null; Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + "included in blockedHashes=%s", received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); @@ -1018,8 +1057,12 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { count -> assertThat(count.get()).isGreaterThan(0)); } + private DrainingHashesTracker getDrainingHashesTracker(Dispatcher dispatcher) { + return ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher).getDrainingHashesTracker(); + } + @Test(dataProvider = "partitioned") - public void testOrderingWithConsumerListener(boolean partitioned) throws Exception { + public void testOrderingWithConsumerListener(KeySharedImplementationType impl, boolean partitioned) throws Exception { final String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); if (partitioned) { admin.topics().createPartitionedTopic(topic, 3); @@ -1075,8 +1118,8 @@ public void testOrderingWithConsumerListener(boolean partitioned) throws Excepti consumer.close(); } - @Test - public void testKeySharedConsumerWithEncrypted() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testKeySharedConsumerWithEncrypted(KeySharedImplementationType impl) throws Exception { final String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); final int totalMessages = 100; @@ -1142,7 +1185,7 @@ public void testKeySharedConsumerWithEncrypted() throws Exception { } @Test(dataProvider = "topicDomain") - public void testSelectorChangedAfterAllConsumerDisconnected(String topicDomain) throws PulsarClientException, + public void testSelectorChangedAfterAllConsumerDisconnected(KeySharedImplementationType impl, String topicDomain) throws PulsarClientException, ExecutionException, InterruptedException { final String topicName = TopicName.get(topicDomain, "public", "default", "testSelectorChangedAfterAllConsumerDisconnected" + UUID.randomUUID()).toString(); @@ -1187,8 +1230,8 @@ public void testSelectorChangedAfterAllConsumerDisconnected(String topicDomain) consumer1.close(); } - @Test - public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected(KeySharedImplementationType impl) throws Exception { final String topicName = "persistent://public/default/change-allow-ooo-delivery-" + UUID.randomUUID(); final String subName = "my-sub"; @@ -1207,7 +1250,7 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr producer.send("message".getBytes()); Awaitility.await().untilAsserted(() -> assertNotNull(consumer1.receive(100, TimeUnit.MILLISECONDS))); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topicName, subName); + StickyKeyDispatcher dispatcher = getDispatcher(topicName, subName); assertTrue(dispatcher.isAllowOutOfOrderDelivery()); consumer1.close(); @@ -1225,8 +1268,8 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr consumer2.close(); } - @Test(timeOut = 30_000) - public void testCheckConsumersWithSameName() throws Exception { + @Test(timeOut = 30_000, dataProvider = "currentImplementationType") + public void testCheckConsumersWithSameName(KeySharedImplementationType impl) throws Exception { final String topicName = "persistent://public/default/same-name-" + UUID.randomUUID(); final String subName = "my-sub"; final String consumerName = "name"; @@ -1270,25 +1313,37 @@ public void testCheckConsumersWithSameName() throws Exception { @Cleanup("shutdownNow") ExecutorService e = Executors.newCachedThreadPool(); e.submit(() -> { - while (l.getCount() > 0) { + while (l.getCount() > 0 && !Thread.currentThread().isInterrupted()) { try { Message msg = c2.receive(1, TimeUnit.SECONDS); + if (msg == null) { + continue; + } c2.acknowledge(msg); l.countDown(); } catch (PulsarClientException ex) { ex.printStackTrace(); + if (ex instanceof PulsarClientException.AlreadyClosedException) { + break; + } } } }); e.submit(() -> { - while (l.getCount() > 0) { + while (l.getCount() > 0 && !Thread.currentThread().isInterrupted()) { try { Message msg = c3.receive(1, TimeUnit.SECONDS); + if (msg == null) { + continue; + } c3.acknowledge(msg); l.countDown(); } catch (PulsarClientException ex) { ex.printStackTrace(); + if (ex instanceof PulsarClientException.AlreadyClosedException) { + break; + } } } }); @@ -1303,7 +1358,7 @@ private Object[][] preSendProvider() { private KeySharedMode getKeySharedModeOfSubscription(Topic topic, String subscription) { if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.persistent)) { - return ((PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subscription) + return ((StickyKeyDispatcher) topic.getSubscription(subscription) .getDispatcher()).getKeySharedMode(); } else if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.non_persistent)) { return ((NonPersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subscription) @@ -1390,45 +1445,37 @@ private void receive(List> consumers) throws PulsarClientException { */ private void receiveAndCheckDistribution(List> consumers, int expectedTotalMessage) throws PulsarClientException { // Add a key so that we know this key was already assigned to one consumer - Map> keyToConsumer = new HashMap<>(); - Map, Integer> messagesPerConsumer = new HashMap<>(); + Map> keyToConsumer = new ConcurrentHashMap<>(); + Map, AtomicInteger> messagesPerConsumer = new ConcurrentHashMap<>(); + AtomicInteger totalMessages = new AtomicInteger(); - int totalMessages = 0; + BiFunction, Message, Boolean> messageHandler = (consumer, msg) -> { + totalMessages.incrementAndGet(); + messagesPerConsumer.computeIfAbsent(consumer, k -> new AtomicInteger()).incrementAndGet(); + try { + consumer.acknowledge(msg); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } - for (Consumer c : consumers) { - int messagesForThisConsumer = 0; - while (true) { - Message msg = c.receive(100, TimeUnit.MILLISECONDS); - if (msg == null) { - // Go to next consumer - messagesPerConsumer.put(c, messagesForThisConsumer); - break; - } - - ++totalMessages; - ++messagesForThisConsumer; - c.acknowledge(msg); - - if (msg.hasKey() || msg.hasOrderingKey()) { - String key = msg.hasOrderingKey() ? new String(msg.getOrderingKey()) : msg.getKey(); - Consumer assignedConsumer = keyToConsumer.get(key); - if (assignedConsumer == null) { - // This is a new key - keyToConsumer.put(key, c); - } else { - // The consumer should be the same - assertEquals(c, assignedConsumer); - } + if (msg.hasKey() || msg.hasOrderingKey()) { + String key = msg.hasOrderingKey() ? new String(msg.getOrderingKey()) : msg.getKey(); + Consumer assignedConsumer = keyToConsumer.putIfAbsent(key, consumer); + if (assignedConsumer != null && !assignedConsumer.equals(consumer)) { + assertEquals(consumer, assignedConsumer); } } - } + return true; + }; - final double PERCENT_ERROR = 0.40; // 40 % + BrokerTestUtil.receiveMessagesInThreads(messageHandler, Duration.ofMillis(250), + consumers.stream().map(Consumer.class::cast)); - double expectedMessagesPerConsumer = totalMessages / consumers.size(); - Assert.assertEquals(expectedTotalMessage, totalMessages); - for (int count : messagesPerConsumer.values()) { - Assert.assertEquals(count, expectedMessagesPerConsumer, expectedMessagesPerConsumer * PERCENT_ERROR); + final double PERCENT_ERROR = 0.40; // 40 % + double expectedMessagesPerConsumer = totalMessages.get() / (double) consumers.size(); + Assert.assertEquals(expectedTotalMessage, totalMessages.get()); + for (AtomicInteger count : messagesPerConsumer.values()) { + Assert.assertEquals(count.get(), expectedMessagesPerConsumer, expectedMessagesPerConsumer * PERCENT_ERROR); } } @@ -1531,8 +1578,8 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe } } - @Test - public void testStickyKeyRangesRestartConsumers() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testStickyKeyRangesRestartConsumers(KeySharedImplementationType impl) throws Exception { final String topic = TopicName.get("persistent", "public", "default", "testStickyKeyRangesRestartConsumers" + UUID.randomUUID()).toString(); @@ -1663,8 +1710,8 @@ public void testStickyKeyRangesRestartConsumers() throws Exception { producerFuture.get(); } - @Test - public void testContinueDispatchMessagesWhenMessageDelayed() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testContinueDispatchMessagesWhenMessageDelayed(KeySharedImplementationType impl) throws Exception { int delayedMessages = 40; int messages = 40; int sum = 0; @@ -1765,8 +1812,8 @@ private AtomicInteger injectReplayReadCounter(String topicName, String cursorNam return replyReadCounter; } - @Test - public void testNoRepeatedReadAndDiscard() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testNoRepeatedReadAndDiscard(KeySharedImplementationType impl) throws Exception { int delayedMessages = 100; int numberOfKeys = delayedMessages; final String topic = newUniqueName("persistent://public/default/tp"); @@ -1839,10 +1886,10 @@ public void testNoRepeatedReadAndDiscard() throws Exception { @DataProvider(name = "allowKeySharedOutOfOrder") public Object[][] allowKeySharedOutOfOrder() { - return new Object[][]{ + return prependImplementationTypeToData(new Object[][]{ {true}, {false} - }; + }); } /** @@ -1860,7 +1907,7 @@ public Object[][] allowKeySharedOutOfOrder() { * - at last, all messages will be received. */ @Test(timeOut = 180 * 1000, dataProvider = "allowKeySharedOutOfOrder") // the test will be finished in 60s. - public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedOutOfOrder) throws Exception { + public void testRecentJoinedPosWillNotStuckOtherConsumer(KeySharedImplementationType impl, boolean allowKeySharedOutOfOrder) throws Exception { final int messagesSentPerTime = 100; final Set totalReceivedMessages = new TreeSet<>(); final String topic = newUniqueName("persistent://public/default/tp"); @@ -2020,8 +2067,10 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO admin.topics().delete(topic, false); } - @Test - public void testReadAheadLimit() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testReadAheadLimit(KeySharedImplementationType impl) throws Exception { + // skip for classic implementation since the feature is not implemented + impl.skipIfClassic(); String topic = "testReadAheadLimit-" + UUID.randomUUID(); int numberOfKeys = 1000; long pauseTime = 100L; @@ -2040,7 +2089,7 @@ public void testReadAheadLimit() throws Exception { .subscribe() .close(); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subscriptionName); + StickyKeyDispatcher dispatcher = getDispatcher(topic, subscriptionName); // create a function to use for checking the number of messages in replay Runnable checkLimit = () -> { @@ -2145,16 +2194,18 @@ public void testReadAheadLimit() throws Exception { private StickyKeyConsumerSelector getSelector(String topic, String subscription) { Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + StickyKeyDispatcher dispatcher = (StickyKeyDispatcher) sub.getDispatcher(); return dispatcher.getSelector(); } // This test case simulates a rolling restart scenario with behaviors that can trigger out-of-order issues. // In earlier versions of Pulsar, this issue occurred in about 25% of cases. // To increase the probability of reproducing the issue, use the invocationCount parameter. - @Test//(invocationCount = 50) - public void testOrderingAfterReconnects() throws Exception { + @Test(dataProvider = "currentImplementationType")//(invocationCount = 50) + public void testOrderingAfterReconnects(KeySharedImplementationType impl) throws Exception { + // skip for classic implementation since this fails + impl.skipIfClassic(); + String topic = newUniqueName("testOrderingAfterReconnects"); int numberOfKeys = 1000; long pauseTime = 100L; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index 02de11a2bcc95..ce554ab2d9c00 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -93,7 +93,7 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -180,7 +180,7 @@ public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(Subscriptio DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -264,7 +264,7 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -356,7 +356,7 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -442,7 +442,7 @@ private void testDispatchRate(SubscriptionType subscription, DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -649,7 +649,7 @@ public void testRateLimitingMultipleConsumers() throws Exception { DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -805,7 +805,7 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName1).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -855,7 +855,7 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { .subscribe(); subDispatcher = topic2.getSubscription(subName2).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java index 390e81ad664f9..8fe1f3e58d96d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java @@ -32,6 +32,7 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -104,8 +105,8 @@ private void triggerNewReadMoreEntries(String tpName, String cursorName) throws PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get(); Dispatcher dispatcher = persistentTopic.getSubscription(cursorName).getDispatcher(); - if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { - ((PersistentDispatcherMultipleConsumers) dispatcher).readMoreEntries(); + if (dispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).readMoreEntriesAsync(); } else if (dispatcher instanceof PersistentDispatcherSingleActiveConsumer) { PersistentDispatcherSingleActiveConsumer persistentDispatcherSingleActiveConsumer = ((PersistentDispatcherSingleActiveConsumer) dispatcher); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionMaxUnackedMessagesTest.java similarity index 85% rename from pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java rename to pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionMaxUnackedMessagesTest.java index 704af89777f05..b3ef641ca1979 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionMaxUnackedMessagesTest.java @@ -34,8 +34,8 @@ import java.util.concurrent.atomic.AtomicLong; import lombok.SneakyThrows; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.Consumer; @@ -47,19 +47,39 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.tests.KeySharedImplementationType; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker-impl") -public class KeySharedSubscriptionTest extends ProducerConsumerBase { +public class KeySharedSubscriptionMaxUnackedMessagesTest extends ProducerConsumerBase { + private final KeySharedImplementationType implementationType; + + // Comment out the next line (Factory annotation) to run tests manually in IntelliJ, one-by-one + @Factory + public static Object[] createTestInstances() { + return KeySharedImplementationType.generateTestInstances(KeySharedSubscriptionMaxUnackedMessagesTest::new); + } + + public KeySharedSubscriptionMaxUnackedMessagesTest() { + // set the default implementation type for manual running in IntelliJ + this(KeySharedImplementationType.DEFAULT); + } + + public KeySharedSubscriptionMaxUnackedMessagesTest(KeySharedImplementationType implementationType) { + this.implementationType = implementationType; + } @Override @BeforeMethod protected void setup() throws Exception { + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(implementationType.classic); + conf.setSubscriptionSharedUseClassicPersistentImplementation(implementationType.classic); conf.setMaxUnackedMessagesPerConsumer(10); super.internalSetup(); super.producerBaseSetup(); @@ -82,16 +102,17 @@ enum KeySharedSelectorType { @DataProvider public Object[][] subType() { - return new Object[][] { - { SubscriptionType.Shared, null }, - { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_ConsistentHashing }, - { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_Classic }, - { SubscriptionType.Key_Shared, KeySharedSelectorType.Sticky } - }; + return implementationType.prependImplementationTypeToData(new Object[][]{ + {SubscriptionType.Shared, null}, + {SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_ConsistentHashing}, + {SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_Classic}, + {SubscriptionType.Key_Shared, KeySharedSelectorType.Sticky} + }); } @Test(dataProvider = "subType", timeOut = 30000) - public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType, + public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(KeySharedImplementationType impl, + SubscriptionType subscriptionType, KeySharedSelectorType selectorType) throws PulsarClientException { if (selectorType == KeySharedSelectorType.AutoSplit_Classic) { @@ -258,8 +279,7 @@ private static void waitUntilLastActiveTimeNoLongerGetsUpdated(AtomicLong lastAc private StickyKeyConsumerSelector getSelector(String topic, String subscription) { Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + StickyKeyDispatcher dispatcher = (StickyKeyDispatcher) sub.getDispatcher(); return dispatcher.getSelector(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java b/pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java new file mode 100644 index 0000000000000..39b504131fcee --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java @@ -0,0 +1,61 @@ +/* + * 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.tests; + +import java.util.Arrays; +import java.util.function.Function; +import org.testng.SkipException; + +/** + * KeyShared implementation type used in test. + */ +public enum KeySharedImplementationType { + // default implementation, PIP-379 + PIP379(false), + // classic implementation before PIP-282 and PIP-379 + Classic(true); + + public static final KeySharedImplementationType DEFAULT = PIP379; + public final boolean classic; + + KeySharedImplementationType(boolean classic) { + this.classic = classic; + } + + public void skipIfClassic() { + if (classic) { + throw new SkipException("Test is not applicable for classic implementation"); + } + } + + public Object[][] prependImplementationTypeToData(Object[][] data) { + return Arrays.stream(data) + .map(array -> { + Object[] newArray = new Object[array.length + 1]; + newArray[0] = this; + System.arraycopy(array, 0, newArray, 1, array.length); + return newArray; + }) + .toArray(Object[][]::new); + } + + public static Object[] generateTestInstances(Function testInstanceFactory) { + return Arrays.stream(KeySharedImplementationType.values()).map(testInstanceFactory).toArray(); + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java index 5f2cf7b209ee9..d2d3600df96ed 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java @@ -72,8 +72,8 @@ public interface ConsumerStats { /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ boolean isBlockedConsumerOnUnackedMsgs(); - /** The last sent position of the cursor when the consumer joining. */ - String getLastSentPositionWhenJoining(); + /** The read position of the cursor when the consumer joining. */ + String getReadPositionWhenJoining(); /** Address of this consumer. */ String getAddress(); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index 7b7c1f5765cc5..ce3a080a855da 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -118,6 +118,9 @@ public interface SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ String getKeySharedMode(); + /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ + Map getConsumersAfterMarkDeletePosition(); + /** SubscriptionProperties (key/value strings) associated with this subscribe. */ Map getSubscriptionProperties(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java index b4c5d21e6926e..de36b330b7f1a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java @@ -77,8 +77,8 @@ public class ConsumerStatsImpl implements ConsumerStats { /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ public boolean blockedConsumerOnUnackedMsgs; - /** The last sent position of the cursor when the consumer joining. */ - public String lastSentPositionWhenJoining; + /** The read position of the cursor when the consumer joining. */ + public String readPositionWhenJoining; /** Address of this consumer. */ private String address; @@ -113,7 +113,7 @@ public ConsumerStatsImpl add(ConsumerStatsImpl stats) { this.availablePermits += stats.availablePermits; this.unackedMessages += stats.unackedMessages; this.blockedConsumerOnUnackedMsgs = stats.blockedConsumerOnUnackedMsgs; - this.lastSentPositionWhenJoining = stats.lastSentPositionWhenJoining; + this.readPositionWhenJoining = stats.readPositionWhenJoining; return this; } @@ -141,8 +141,8 @@ public void setClientVersion(String clientVersion) { this.clientVersion = clientVersion; } - public String getLastSentPositionWhenJoining() { - return lastSentPositionWhenJoining; + public String getReadPositionWhenJoining() { + return readPositionWhenJoining; } public String getLastAckedTime() { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index 4206a4aa8d61b..12734a5586cef 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.ArrayList; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -125,8 +126,8 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ public String keySharedMode; - /** The last sent position of the cursor. This is for Key_Shared subscription. */ - public String lastSentPosition; + /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ + public Map consumersAfterMarkDeletePosition; /** The number of non-contiguous deleted messages ranges. */ public int nonContiguousDeletedMessagesRanges; @@ -153,6 +154,7 @@ public class SubscriptionStatsImpl implements SubscriptionStats { public SubscriptionStatsImpl() { this.consumers = new ArrayList<>(); + this.consumersAfterMarkDeletePosition = new LinkedHashMap<>(); this.subscriptionProperties = new HashMap<>(); this.bucketDelayedIndexStats = new HashMap<>(); } @@ -177,6 +179,7 @@ public void reset() { lastExpireTimestamp = 0L; lastMarkDeleteAdvancedTimestamp = 0L; consumers.clear(); + consumersAfterMarkDeletePosition.clear(); nonContiguousDeletedMessagesRanges = 0; nonContiguousDeletedMessagesRangesSerializedSize = 0; earliestMsgPublishTimeInBacklog = 0L; @@ -222,6 +225,7 @@ public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { } } this.allowOutOfOrderDelivery |= stats.allowOutOfOrderDelivery; + this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; if (this.earliestMsgPublishTimeInBacklog != 0 && stats.earliestMsgPublishTimeInBacklog != 0) {