diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt
index 1d78913849bda..1ebf2229b69d4 100644
--- a/distribution/server/src/assemble/LICENSE.bin.txt
+++ b/distribution/server/src/assemble/LICENSE.bin.txt
@@ -262,6 +262,7 @@ The Apache Software License, Version 2.0
- com.fasterxml.jackson.module-jackson-module-parameter-names-2.17.2.jar
* Caffeine -- com.github.ben-manes.caffeine-caffeine-2.9.1.jar
* Conscrypt -- org.conscrypt-conscrypt-openjdk-uber-2.5.2.jar
+ * Fastutil -- it.unimi.dsi-fastutil-8.5.14.jar
* Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.17.0.jar
* Bitbucket -- org.bitbucket.b_c-jose4j-0.9.4.jar
* Gson
diff --git a/pom.xml b/pom.xml
index 881a1541c5eaf..a248adbb5675f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -162,6 +162,7 @@ flexible messaging model and an intuitive client API.
1.0.7
1.0.2.5
2.17.2
+ 8.5.14
0.10.2
1.6.2
10.14.2
@@ -911,6 +912,12 @@ flexible messaging model and an intuitive client API.
import
+
+ it.unimi.dsi
+ fastutil
+ ${fastutil.version}
+
+
org.codehaus.jettison
jettison
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 33b4fbff5f5bb..5877310fed23b 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
@@ -979,6 +979,16 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece
)
private int keySharedLookAheadMsgInReplayThresholdPerSubscription = 20000;
+ @FieldContext(
+ category = CATEGORY_POLICIES,
+ doc = "For Key_Shared subscriptions, when a blocked key hash gets unblocked,"
+ + " a redelivery will be attempted after a delay. This setting controls the delay."
+ + " The reason to have the delay is to batch multiple unblocking events instead of triggering"
+ + " redelivery for each unblocking event.",
+ dynamic = true
+ )
+ private long keySharedUnblockingIntervalMs = 10L;
+
@FieldContext(
category = CATEGORY_POLICIES,
doc = "Once broker reaches maxUnackedMessagesPerBroker limit, it blocks subscriptions which has higher "
diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml
index ee22762719175..a9521e76296de 100644
--- a/pulsar-broker/pom.xml
+++ b/pulsar-broker/pom.xml
@@ -63,6 +63,11 @@
protobuf-java
+
+ it.unimi.dsi
+ fastutil
+
+
${project.groupId}
pulsar-client-original
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 5c99e4c307d7c..f93a627bca7b8 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
@@ -146,7 +146,7 @@ public void run(Timeout timeout) throws Exception {
lastTickRun = clock.millis();
currentTimeoutTarget = -1;
this.timeout = null;
- dispatcher.readMoreEntries();
+ dispatcher.readMoreEntriesAsync();
}
}
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 bfa99eedcadce..d5f82955f690d 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
@@ -3351,7 +3351,7 @@ public void unblockDispatchersOnUnAckMessages(List {
dispatcher.unBlockDispatcherOnUnackedMsgs();
- executor().execute(() -> dispatcher.readMoreEntries());
+ dispatcher.readMoreEntriesAsync();
log.info("[{}] Dispatcher is unblocked", dispatcher.getName());
blockedDispatchers.remove(dispatcher);
});
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java
index 1ae9a6ff96b7d..8381f9543bdc2 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java
@@ -19,7 +19,6 @@
package org.apache.pulsar.broker.service;
import java.util.ArrayList;
-import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
@@ -28,13 +27,10 @@
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.pulsar.client.api.Range;
-import org.apache.pulsar.common.util.Murmur3_32Hash;
/**
- * This is a consumer selector based fixed hash range.
- *
- * The implementation uses consistent hashing to evenly split, the
- * number of keys assigned to each consumer.
+ * This is a consumer selector using consistent hashing to evenly split
+ * the number of keys assigned to each consumer.
*/
public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyConsumerSelector {
// use NUL character as field separator for hash key calculation
@@ -47,14 +43,22 @@ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyCons
private final ConsumerNameIndexTracker consumerNameIndexTracker = new ConsumerNameIndexTracker();
private final int numberOfPoints;
+ private final Range keyHashRange;
+ private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot;
public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints) {
+ this(numberOfPoints, DEFAULT_RANGE_SIZE - 1);
+ }
+
+ public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints, int rangeMaxValue) {
this.hashRing = new TreeMap<>();
this.numberOfPoints = numberOfPoints;
+ this.keyHashRange = Range.of(STICKY_KEY_HASH_NOT_SET + 1, rangeMaxValue);
+ this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty();
}
@Override
- public CompletableFuture addConsumer(Consumer consumer) {
+ public CompletableFuture addConsumer(Consumer consumer) {
rwLock.writeLock().lock();
try {
ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer);
@@ -72,7 +76,11 @@ public CompletableFuture addConsumer(Consumer consumer) {
consumerNameIndexTracker.decreaseConsumerRefCount(removed);
}
}
- return CompletableFuture.completedFuture(null);
+ ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot();
+ ImpactedConsumersResult impactedConsumers =
+ consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter);
+ consumerHashAssignmentsSnapshot = assignmentsAfter;
+ return CompletableFuture.completedFuture(impactedConsumers);
} finally {
rwLock.writeLock().unlock();
}
@@ -88,14 +96,14 @@ public CompletableFuture addConsumer(Consumer consumer) {
* @param hashRingPointIndex the index of the hash ring point
* @return the hash value
*/
- private static int calculateHashForConsumerAndIndex(Consumer consumer, int consumerNameIndex,
+ private int calculateHashForConsumerAndIndex(Consumer consumer, int consumerNameIndex,
int hashRingPointIndex) {
String key = consumer.consumerName() + KEY_SEPARATOR + consumerNameIndex + KEY_SEPARATOR + hashRingPointIndex;
- return Murmur3_32Hash.getInstance().makeHash(key.getBytes());
+ return makeStickyKeyHash(key.getBytes());
}
@Override
- public void removeConsumer(Consumer consumer) {
+ public ImpactedConsumersResult removeConsumer(Consumer consumer) {
rwLock.writeLock().lock();
try {
ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer);
@@ -109,6 +117,11 @@ public void removeConsumer(Consumer consumer) {
}
}
}
+ ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot();
+ ImpactedConsumersResult impactedConsumers =
+ consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter);
+ consumerHashAssignmentsSnapshot = assignmentsAfter;
+ return impactedConsumers;
} finally {
rwLock.writeLock().unlock();
}
@@ -134,32 +147,58 @@ public Consumer select(int hash) {
}
@Override
- public Map> getConsumerKeyHashRanges() {
- Map> result = new IdentityHashMap<>();
+ public Range getKeyHashRange() {
+ return keyHashRange;
+ }
+
+ @Override
+ public ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() {
rwLock.readLock().lock();
try {
- if (hashRing.isEmpty()) {
- return result;
- }
- int start = 0;
- int lastKey = 0;
- for (Map.Entry entry: hashRing.entrySet()) {
- Consumer consumer = entry.getValue().consumer;
- result.computeIfAbsent(consumer, key -> new ArrayList<>())
- .add(Range.of(start, entry.getKey()));
- lastKey = entry.getKey();
- start = lastKey + 1;
- }
- // Handle wrap-around in the hash ring, the first consumer will also contain the range from the last key
- // to the maximum value of the hash range
- Consumer firstConsumer = hashRing.firstEntry().getValue().consumer;
- List ranges = result.get(firstConsumer);
- if (lastKey != Integer.MAX_VALUE - 1) {
- ranges.add(Range.of(lastKey + 1, Integer.MAX_VALUE - 1));
- }
+ return consumerHashAssignmentsSnapshot;
} finally {
rwLock.readLock().unlock();
}
- return result;
+ }
+
+ private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() {
+ if (hashRing.isEmpty()) {
+ return ConsumerHashAssignmentsSnapshot.empty();
+ }
+ List result = new ArrayList<>();
+ int start = getKeyHashRange().getStart();
+ int lastKey = -1;
+ Consumer previousConsumer = null;
+ Range previousRange = null;
+ for (Map.Entry entry: hashRing.entrySet()) {
+ Consumer consumer = entry.getValue().consumer;
+ Range range;
+ if (consumer == previousConsumer) {
+ // join ranges
+ result.remove(result.size() - 1);
+ range = Range.of(previousRange.getStart(), entry.getKey());
+ } else {
+ range = Range.of(start, entry.getKey());
+ }
+ result.add(new HashRangeAssignment(range, consumer));
+ lastKey = entry.getKey();
+ start = lastKey + 1;
+ previousConsumer = consumer;
+ previousRange = range;
+ }
+ // Handle wrap-around
+ Consumer firstConsumer = hashRing.firstEntry().getValue().consumer;
+ if (lastKey != getKeyHashRange().getEnd()) {
+ Range range;
+ if (firstConsumer == previousConsumer && previousRange.getEnd() == lastKey) {
+ // join ranges
+ result.remove(result.size() - 1);
+ range = Range.of(previousRange.getStart(), getKeyHashRange().getEnd());
+ } else {
+ range = Range.of(lastKey + 1, getKeyHashRange().getEnd());
+ }
+ result.add(new HashRangeAssignment(range, firstConsumer));
+ }
+ return ConsumerHashAssignmentsSnapshot.of(result);
}
}
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 7f46e8969eb53..c9584f2c1790f 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
@@ -19,6 +19,7 @@
package org.apache.pulsar.broker.service;
import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET;
import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.MoreObjects;
@@ -26,6 +27,8 @@
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import io.opentelemetry.api.common.Attributes;
+import it.unimi.dsi.fastutil.ints.IntIntPair;
+import it.unimi.dsi.fastutil.objects.ObjectIntPair;
import java.time.Instant;
import java.util.ArrayList;
import java.util.BitSet;
@@ -65,14 +68,11 @@
import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl;
import org.apache.pulsar.common.policies.data.TopicOperation;
import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl;
-import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.schema.SchemaType;
import org.apache.pulsar.common.stats.Rate;
import org.apache.pulsar.common.util.DateFormatter;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.collections.BitSetRecyclable;
-import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap;
-import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair;
import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes;
import org.apache.pulsar.transaction.common.exception.TransactionConflictException;
import org.slf4j.Logger;
@@ -119,7 +119,7 @@ public class Consumer {
AtomicIntegerFieldUpdater.newUpdater(Consumer.class, "permitsReceivedWhileConsumerBlocked");
private volatile int permitsReceivedWhileConsumerBlocked = 0;
- private final ConcurrentLongLongPairHashMap pendingAcks;
+ private final PendingAcksMap pendingAcks;
private final ConsumerStatsImpl stats;
@@ -167,6 +167,13 @@ public class Consumer {
private static final AtomicReferenceFieldUpdater OPEN_TELEMETRY_ATTRIBUTES_FIELD_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(Consumer.class, Attributes.class, "openTelemetryAttributes");
+ @Getter
+ @Setter
+ private volatile PendingAcksMap.PendingAcksAddHandler pendingAcksAddHandler;
+ @Getter
+ @Setter
+ private volatile PendingAcksMap.PendingAcksRemoveHandler pendingAcksRemoveHandler;
+
public Consumer(Subscription subscription, SubType subType, String topicName, long consumerId,
int priorityLevel, String consumerName,
boolean isDurable, TransportCnx cnx, String appId,
@@ -223,12 +230,8 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo
stats.metadata = this.metadata;
if (Subscription.isIndividualAckMode(subType)) {
- this.pendingAcks = ConcurrentLongLongPairHashMap.newBuilder()
- .autoShrink(subscription.getTopic().getBrokerService()
- .getPulsar().getConfiguration().isAutoShrinkForConsumerPendingAcksMap())
- .expectedItems(256)
- .concurrencyLevel(1)
- .build();
+ this.pendingAcks = new PendingAcksMap(this, this::getPendingAcksAddHandler,
+ this::getPendingAcksRemoveHandler);
} else {
// We don't need to keep track of pending acks if the subscription is not shared
this.pendingAcks = null;
@@ -359,17 +362,43 @@ public Future sendMessages(final List extends Entry> entries,
// because this consumer is possible to disconnect at this time.
if (pendingAcks != null) {
int batchSize = batchSizes.getBatchSize(i);
- int stickyKeyHash = stickyKeyHashes == null ? getStickyKeyHash(entry) : stickyKeyHashes.get(i);
- long[] ackSet = batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i);
- if (ackSet != null) {
- unackedMessages -= (batchSize - BitSet.valueOf(ackSet).cardinality());
+ int stickyKeyHash;
+ if (stickyKeyHashes == null) {
+ if (entry instanceof EntryAndMetadata entryAndMetadata) {
+ stickyKeyHash = entryAndMetadata.getCachedStickyKeyHash();
+ } else {
+ stickyKeyHash = STICKY_KEY_HASH_NOT_SET;
+ }
+ } else {
+ stickyKeyHash = stickyKeyHashes.get(i);
}
- pendingAcks.put(entry.getLedgerId(), entry.getEntryId(), batchSize, stickyKeyHash);
- if (log.isDebugEnabled()) {
- log.debug("[{}-{}] Added {}:{} ledger entry with batchSize of {} to pendingAcks in"
- + " broker.service.Consumer for consumerId: {}",
- topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize,
- consumerId);
+ boolean sendingAllowed =
+ pendingAcks.addPendingAckIfAllowed(entry.getLedgerId(), entry.getEntryId(), batchSize,
+ stickyKeyHash);
+ if (!sendingAllowed) {
+ // sending isn't allowed when pending acks doesn't accept adding the entry
+ // this happens when Key_Shared draining hashes contains the stickyKeyHash
+ // because of race conditions, it might be resolved at the time of sending
+ totalEntries--;
+ entries.set(i, null);
+ entry.release();
+ if (log.isDebugEnabled()) {
+ log.debug("[{}-{}] Skipping sending of {}:{} ledger entry with batchSize of {} since adding"
+ + " to pending acks failed in broker.service.Consumer for consumerId: {}",
+ topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize,
+ consumerId);
+ }
+ } else {
+ long[] ackSet = batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i);
+ if (ackSet != null) {
+ unackedMessages -= (batchSize - BitSet.valueOf(ackSet).cardinality());
+ }
+ if (log.isDebugEnabled()) {
+ log.debug("[{}-{}] Added {}:{} ledger entry with batchSize of {} to pendingAcks in"
+ + " broker.service.Consumer for consumerId: {}",
+ topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize,
+ consumerId);
+ }
}
}
}
@@ -537,11 +566,11 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map ackOwnerConsumerAndBatchSize =
+ ObjectIntPair ackOwnerConsumerAndBatchSize =
getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), msgId.getEntryId());
- Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.getLeft();
+ Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.left();
long ackedCount;
- long batchSize = ackOwnerConsumerAndBatchSize.getRight();
+ int batchSize = ackOwnerConsumerAndBatchSize.rightInt();
if (msgId.getAckSetsCount() > 0) {
long[] ackSets = new long[msgId.getAckSetsCount()];
for (int j = 0; j < msgId.getAckSetsCount(); j++) {
@@ -607,11 +636,17 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) {
for (int i = 0; i < ack.getMessageIdsCount(); i++) {
MessageIdData msgId = ack.getMessageIdAt(i);
Position position = AckSetStateUtil.createPositionWithAckSet(msgId.getLedgerId(), msgId.getEntryId(), null);
- Consumer ackOwnerConsumer = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(),
- msgId.getEntryId()).getLeft();
+ ObjectIntPair ackOwnerConsumerAndBatchSize = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(),
+ msgId.getEntryId());
+ if (ackOwnerConsumerAndBatchSize == null) {
+ log.warn("[{}] [{}] Acknowledging message at {} that was already deleted", subscription,
+ consumerId, position);
+ continue;
+ }
+ Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.left();
// acked count at least one
long ackedCount;
- long batchSize;
+ int batchSize;
if (msgId.hasBatchSize()) {
batchSize = msgId.getBatchSize();
// ack batch messages set ackeCount = batchSize
@@ -660,7 +695,7 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) {
return completableFuture.thenApply(__ -> totalAckCount.sum());
}
- private long getAckedCountForMsgIdNoAckSets(long batchSize, Position position, Consumer consumer) {
+ private long getAckedCountForMsgIdNoAckSets(int batchSize, Position position, Consumer consumer) {
if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType)) {
long[] cursorAckSet = getCursorAckSet(position);
if (cursorAckSet != null) {
@@ -670,11 +705,11 @@ private long getAckedCountForMsgIdNoAckSets(long batchSize, Position position, C
return batchSize;
}
- private long getAckedCountForBatchIndexLevelEnabled(Position position, long batchSize, long[] ackSets,
+ private long getAckedCountForBatchIndexLevelEnabled(Position position, int batchSize, long[] ackSets,
Consumer consumer) {
long ackedCount = 0;
if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType)
- && consumer.getPendingAcks().get(position.getLedgerId(), position.getEntryId()) != null) {
+ && consumer.getPendingAcks().contains(position.getLedgerId(), position.getEntryId())) {
long[] cursorAckSet = getCursorAckSet(position);
if (cursorAckSet != null) {
BitSetRecyclable cursorBitSet = BitSetRecyclable.create().resetWords(cursorAckSet);
@@ -692,14 +727,14 @@ private long getAckedCountForBatchIndexLevelEnabled(Position position, long batc
return ackedCount;
}
- private long getAckedCountForTransactionAck(long batchSize, long[] ackSets) {
+ private long getAckedCountForTransactionAck(int batchSize, long[] ackSets) {
BitSetRecyclable bitset = BitSetRecyclable.create().resetWords(ackSets);
long ackedCount = batchSize - bitset.cardinality();
bitset.recycle();
return ackedCount;
}
- private long getUnAckedCountForBatchIndexLevelEnabled(Position position, long batchSize) {
+ private long getUnAckedCountForBatchIndexLevelEnabled(Position position, int batchSize) {
long unAckedCount = batchSize;
if (isAcknowledgmentAtBatchIndexLevelEnabled) {
long[] cursorAckSet = getCursorAckSet(position);
@@ -734,24 +769,24 @@ private boolean checkCanRemovePendingAcksAndHandle(Consumer ackOwnedConsumer,
* @param entryId The ID of the entry.
* @return Pair
*/
- private Pair getAckOwnerConsumerAndBatchSize(long ledgerId, long entryId) {
+ private ObjectIntPair getAckOwnerConsumerAndBatchSize(long ledgerId, long entryId) {
if (Subscription.isIndividualAckMode(subType)) {
- LongPair longPair = getPendingAcks().get(ledgerId, entryId);
- if (longPair != null) {
- return Pair.of(this, longPair.first);
+ IntIntPair pendingAck = getPendingAcks().get(ledgerId, entryId);
+ if (pendingAck != null) {
+ return ObjectIntPair.of(this, pendingAck.leftInt());
} else {
// If there are more consumers, this step will consume more CPU, and it should be optimized later.
for (Consumer consumer : subscription.getConsumers()) {
if (consumer != this) {
- longPair = consumer.getPendingAcks().get(ledgerId, entryId);
- if (longPair != null) {
- return Pair.of(consumer, longPair.first);
+ pendingAck = consumer.getPendingAcks().get(ledgerId, entryId);
+ if (pendingAck != null) {
+ return ObjectIntPair.of(consumer, pendingAck.leftInt());
}
}
}
}
}
- return Pair.of(this, 1L);
+ return ObjectIntPair.of(this, 1);
}
private long[] getCursorAckSet(Position position) {
@@ -1027,7 +1062,8 @@ public int hashCode() {
* @param position
*/
private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position) {
- if (!ackOwnedConsumer.getPendingAcks().remove(position.getLedgerId(), position.getEntryId())) {
+ PendingAcksMap ownedConsumerPendingAcks = ackOwnedConsumer.getPendingAcks();
+ if (!ownedConsumerPendingAcks.remove(position.getLedgerId(), position.getEntryId())) {
// Message was already removed by the other consumer
return false;
}
@@ -1046,7 +1082,7 @@ private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position)
return true;
}
- public ConcurrentLongLongPairHashMap getPendingAcks() {
+ public PendingAcksMap getPendingAcks() {
return pendingAcks;
}
@@ -1093,9 +1129,9 @@ public void redeliverUnacknowledgedMessages(List messageIds) {
List pendingPositions = new ArrayList<>();
for (MessageIdData msg : messageIds) {
Position position = PositionFactory.create(msg.getLedgerId(), msg.getEntryId());
- LongPair longPair = pendingAcks.get(position.getLedgerId(), position.getEntryId());
- if (longPair != null) {
- int unAckedCount = (int) getUnAckedCountForBatchIndexLevelEnabled(position, longPair.first);
+ IntIntPair pendingAck = pendingAcks.get(position.getLedgerId(), position.getEntryId());
+ if (pendingAck != null) {
+ int unAckedCount = (int) getUnAckedCountForBatchIndexLevelEnabled(position, pendingAck.leftInt());
pendingAcks.remove(position.getLedgerId(), position.getEntryId());
totalRedeliveryMessages += unAckedCount;
pendingPositions.add(position);
@@ -1191,16 +1227,6 @@ public Map getMetadata() {
return metadata;
}
- private int getStickyKeyHash(Entry entry) {
- final byte[] stickyKey;
- if (entry instanceof EntryAndMetadata) {
- stickyKey = ((EntryAndMetadata) entry).getStickyKey();
- } else {
- stickyKey = Commands.peekStickyKey(entry.getDataBuffer(), topicName, subscription.getName());
- }
- return StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey);
- }
-
private static final Logger log = LoggerFactory.getLogger(Consumer.class);
public Attributes getOpenTelemetryAttributes() {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java
new file mode 100644
index 0000000000000..d2bd113e69d1e
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java
@@ -0,0 +1,224 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.client.api.Range;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Represents the hash ranges assigned to each consumer in a {@link StickyKeyConsumerSelector} at a point in time.
+ */
+@EqualsAndHashCode(exclude = "cachedRangesByConsumer")
+@ToString(exclude = "cachedRangesByConsumer")
+public class ConsumerHashAssignmentsSnapshot {
+ private final List hashRangeAssignments;
+ private Map> cachedRangesByConsumer;
+
+ private ConsumerHashAssignmentsSnapshot(List hashRangeAssignments) {
+ validate(hashRangeAssignments);
+ this.hashRangeAssignments = hashRangeAssignments;
+ }
+
+ private void validate(List hashRangeAssignments) {
+ Range previousRange = null;
+ for (HashRangeAssignment hashRangeAssignment : hashRangeAssignments) {
+ Range range = hashRangeAssignment.range();
+ Consumer consumer = hashRangeAssignment.consumer();
+ if (range == null || consumer == null) {
+ throw new IllegalArgumentException("Range and consumer must not be null");
+ }
+ if (previousRange != null && previousRange.compareTo(range) >= 0) {
+ throw new IllegalArgumentException("Ranges must be non-overlapping and sorted");
+ }
+ previousRange = range;
+ }
+ }
+
+ public static ConsumerHashAssignmentsSnapshot of(List hashRangeAssignments) {
+ return new ConsumerHashAssignmentsSnapshot(hashRangeAssignments);
+ }
+
+ public static ConsumerHashAssignmentsSnapshot empty() {
+ return new ConsumerHashAssignmentsSnapshot(Collections.emptyList());
+ }
+
+ public ImpactedConsumersResult resolveImpactedConsumers(ConsumerHashAssignmentsSnapshot other) {
+ return resolveConsumerRemovedHashRanges(this.hashRangeAssignments, other.hashRangeAssignments);
+ }
+
+ /**
+ * Get the ranges assigned to each consumer. The ranges are merged if they are overlapping.
+ * @return the ranges assigned to each consumer
+ */
+ public synchronized Map> getRangesByConsumer() {
+ if (cachedRangesByConsumer == null) {
+ cachedRangesByConsumer = internalGetRangesByConsumer();
+ }
+ return cachedRangesByConsumer;
+ }
+
+ private @NotNull Map> internalGetRangesByConsumer() {
+ Map> rangesByConsumer = new IdentityHashMap<>();
+ hashRangeAssignments.forEach(entry -> {
+ Range range = entry.range();
+ Consumer consumer = entry.consumer();
+ rangesByConsumer.computeIfAbsent(consumer, k -> new TreeSet<>()).add(range);
+ });
+ Map> mergedOverlappingRangesByConsumer = new IdentityHashMap<>();
+ rangesByConsumer.forEach((consumer, ranges) -> {
+ mergedOverlappingRangesByConsumer.put(consumer, mergeOverlappingRanges(ranges));
+ });
+ return mergedOverlappingRangesByConsumer;
+ }
+
+ @VisibleForTesting
+ Map> diffRanges(ConsumerHashAssignmentsSnapshot other) {
+ return diffRanges(this.hashRangeAssignments, other.hashRangeAssignments);
+ }
+
+ /**
+ * Resolve the consumers where the existing range was removed by a change.
+ * @param mappingBefore the range mapping before the change
+ * @param mappingAfter the range mapping after the change
+ * @return consumers and ranges where the existing range changed
+ */
+ static ImpactedConsumersResult resolveConsumerRemovedHashRanges(List mappingBefore,
+ List mappingAfter) {
+ Map> impactedRanges = diffRanges(mappingBefore, mappingAfter);
+ Map> removedRangesByConsumer = impactedRanges.entrySet().stream()
+ .collect(IdentityHashMap::new, (resultMap, entry) -> {
+ Range range = entry.getKey();
+ // filter out only where the range was removed
+ Consumer consumerBefore = entry.getValue().getLeft();
+ if (consumerBefore != null) {
+ resultMap.computeIfAbsent(consumerBefore, k -> new TreeSet<>()).add(range);
+ }
+ }, IdentityHashMap::putAll);
+ return mergedOverlappingRangesAndConvertToImpactedConsumersResult(removedRangesByConsumer);
+ }
+
+ static ImpactedConsumersResult mergedOverlappingRangesAndConvertToImpactedConsumersResult(
+ Map> removedRangesByConsumer) {
+ Map mergedRangesByConsumer = new IdentityHashMap<>();
+ removedRangesByConsumer.forEach((consumer, ranges) -> {
+ mergedRangesByConsumer.put(consumer, RemovedHashRanges.of(mergeOverlappingRanges(ranges)));
+ });
+ return ImpactedConsumersResult.of(mergedRangesByConsumer);
+ }
+
+ /**
+ * Merge overlapping ranges.
+ * @param ranges the ranges to merge
+ * @return the merged ranges
+ */
+ static List mergeOverlappingRanges(SortedSet ranges) {
+ List mergedRanges = new ArrayList<>();
+ Iterator rangeIterator = ranges.iterator();
+ Range currentRange = rangeIterator.hasNext() ? rangeIterator.next() : null;
+ while (rangeIterator.hasNext()) {
+ Range nextRange = rangeIterator.next();
+ if (currentRange.getEnd() >= nextRange.getStart() - 1) {
+ currentRange = Range.of(currentRange.getStart(), Math.max(currentRange.getEnd(), nextRange.getEnd()));
+ } else {
+ mergedRanges.add(currentRange);
+ currentRange = nextRange;
+ }
+ }
+ if (currentRange != null) {
+ mergedRanges.add(currentRange);
+ }
+ return mergedRanges;
+ }
+
+ /**
+ * Calculate the diff of two range mappings.
+ * @param mappingBefore the range mapping before
+ * @param mappingAfter the range mapping after
+ * @return the impacted ranges where the consumer is changed from the before to the after
+ */
+ static Map> diffRanges(List mappingBefore,
+ List mappingAfter) {
+ Map> impactedRanges = new LinkedHashMap<>();
+ Iterator beforeIterator = mappingBefore.iterator();
+ Iterator afterIterator = mappingAfter.iterator();
+
+ HashRangeAssignment beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null;
+ HashRangeAssignment afterEntry = afterIterator.hasNext() ? afterIterator.next() : null;
+
+ while (beforeEntry != null && afterEntry != null) {
+ Range beforeRange = beforeEntry.range();
+ Range afterRange = afterEntry.range();
+ Consumer beforeConsumer = beforeEntry.consumer();
+ Consumer afterConsumer = afterEntry.consumer();
+
+ if (beforeRange.equals(afterRange)) {
+ if (!beforeConsumer.equals(afterConsumer)) {
+ impactedRanges.put(afterRange, Pair.of(beforeConsumer, afterConsumer));
+ }
+ beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null;
+ afterEntry = afterIterator.hasNext() ? afterIterator.next() : null;
+ } else if (beforeRange.getEnd() < afterRange.getStart()) {
+ impactedRanges.put(beforeRange, Pair.of(beforeConsumer, afterConsumer));
+ beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null;
+ } else if (afterRange.getEnd() < beforeRange.getStart()) {
+ impactedRanges.put(afterRange, Pair.of(beforeConsumer, afterConsumer));
+ afterEntry = afterIterator.hasNext() ? afterIterator.next() : null;
+ } else {
+ Range overlapRange = Range.of(
+ Math.max(beforeRange.getStart(), afterRange.getStart()),
+ Math.min(beforeRange.getEnd(), afterRange.getEnd())
+ );
+ if (!beforeConsumer.equals(afterConsumer)) {
+ impactedRanges.put(overlapRange, Pair.of(beforeConsumer, afterConsumer));
+ }
+ if (beforeRange.getEnd() <= overlapRange.getEnd()) {
+ beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null;
+ }
+ if (afterRange.getEnd() <= overlapRange.getEnd()) {
+ afterEntry = afterIterator.hasNext() ? afterIterator.next() : null;
+ }
+ }
+ }
+
+ while (beforeEntry != null) {
+ impactedRanges.put(beforeEntry.range(), Pair.of(beforeEntry.consumer(), null));
+ beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null;
+ }
+
+ while (afterEntry != null) {
+ impactedRanges.put(afterEntry.range(), Pair.of(null, afterEntry.consumer()));
+ afterEntry = afterIterator.hasNext() ? afterIterator.next() : null;
+ }
+
+ return impactedRanges;
+ }
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java
new file mode 100644
index 0000000000000..3521fa197a13d
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java
@@ -0,0 +1,241 @@
+/*
+ * 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 static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * A thread-safe map to store draining hashes in the consumer.
+ */
+@Slf4j
+public class DrainingHashesTracker {
+ private final String dispatcherName;
+ private final UnblockingHandler unblockingHandler;
+ // optimize the memory consumption of the map by using primitive int keys
+ private final Int2ObjectOpenHashMap drainingHashes = new Int2ObjectOpenHashMap<>();
+ int batchLevel;
+ boolean unblockedWhileBatching;
+
+ /**
+ * Represents an entry in the draining hashes tracker.
+ */
+ @ToString
+ public static class DrainingHashEntry {
+ private final Consumer consumer;
+ private int refCount;
+ private int blockedCount;
+
+ /**
+ * Constructs a new DrainingHashEntry with the specified Consumer.
+ *
+ * @param consumer the Consumer instance
+ */
+ DrainingHashEntry(Consumer consumer) {
+ this.consumer = consumer;
+ }
+
+ /**
+ * Gets the consumer that contained the hash in pending acks at the time of creating this
+ * entry. Since a particular hash can be assigned to only one consumer at a time, this consumer
+ * cannot change. No new pending acks can be added in the {@link PendingAcksMap} when there's
+ * a draining hash entry for a hash in {@link DrainingHashesTracker}.
+ *
+ * @return the consumer instance that contained the hash in pending acks at the time of creating this entry
+ */
+ public Consumer getConsumer() {
+ return consumer;
+ }
+
+ /**
+ * Increments the reference count.
+ */
+ void incrementRefCount() {
+ refCount++;
+ }
+
+ /**
+ * Decrements the reference count.
+ *
+ * @return true if the reference count is zero, false otherwise
+ */
+ boolean decrementRefCount() {
+ return --refCount == 0;
+ }
+
+ /**
+ * Increments the blocked count.
+ */
+ void incrementBlockedCount() {
+ blockedCount++;
+ }
+
+ /**
+ * Checks if the entry is blocking.
+ *
+ * @return true if the blocked count is greater than zero, false otherwise
+ */
+ boolean isBlocking() {
+ return blockedCount > 0;
+ }
+ }
+
+ /**
+ * Interface for handling the unblocking of sticky key hashes.
+ */
+ public interface UnblockingHandler {
+ /**
+ * Handle the unblocking of a sticky key hash.
+ *
+ * @param stickyKeyHash the sticky key hash that has been unblocked, or -1 if hash unblocking is done in batch
+ */
+ void stickyKeyHashUnblocked(int stickyKeyHash);
+ }
+
+ public DrainingHashesTracker(String dispatcherName, UnblockingHandler unblockingHandler) {
+ this.dispatcherName = dispatcherName;
+ this.unblockingHandler = unblockingHandler;
+ }
+
+ /**
+ * Add an entry to the draining hashes tracker.
+ *
+ * @param consumer the consumer
+ * @param stickyHash the sticky hash
+ */
+ public synchronized void addEntry(Consumer consumer, int stickyHash) {
+ if (stickyHash == 0) {
+ throw new IllegalArgumentException("Sticky hash cannot be 0");
+ }
+ DrainingHashEntry entry = drainingHashes.get(stickyHash);
+ if (entry == null) {
+ entry = new DrainingHashEntry(consumer);
+ drainingHashes.put(stickyHash, entry);
+ } else if (entry.getConsumer() != consumer) {
+ throw new IllegalStateException(
+ "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash
+ + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer
+ + ".");
+ }
+ entry.incrementRefCount();
+ }
+
+ /**
+ * Start a batch operation. There could be multiple nested batch operations.
+ * The unblocking of sticky key hashes will be done only when the last batch operation ends.
+ */
+ public synchronized void startBatch() {
+ batchLevel++;
+ }
+
+ /**
+ * End a batch operation.
+ */
+ public synchronized void endBatch() {
+ if (--batchLevel == 0 && unblockedWhileBatching) {
+ unblockedWhileBatching = false;
+ unblockingHandler.stickyKeyHashUnblocked(-1);
+ }
+ }
+
+ /**
+ * Reduce the reference count for a given sticky hash.
+ *
+ * @param consumer the consumer
+ * @param stickyHash the sticky hash
+ * @param closing
+ */
+ public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boolean closing) {
+ if (stickyHash == 0) {
+ return;
+ }
+ DrainingHashEntry entry = drainingHashes.get(stickyHash);
+ if (entry == null) {
+ return;
+ }
+ if (entry.getConsumer() != consumer) {
+ throw new IllegalStateException(
+ "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash
+ + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer
+ + ".");
+ }
+ if (entry.decrementRefCount()) {
+ DrainingHashEntry removed = drainingHashes.remove(stickyHash);
+ if (!closing && removed.isBlocking()) {
+ if (batchLevel > 0) {
+ unblockedWhileBatching = true;
+ } else {
+ unblockingHandler.stickyKeyHashUnblocked(stickyHash);
+ }
+ }
+ }
+ }
+
+ /**
+ * Check if a sticky key hash should be blocked.
+ *
+ * @param consumer the consumer
+ * @param stickyKeyHash the sticky key hash
+ * @return true if the sticky key hash should be blocked, false otherwise
+ */
+ public synchronized boolean shouldBlockStickyKeyHash(Consumer consumer, int stickyKeyHash) {
+ if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) {
+ log.warn("[{}] Sticky key hash is not set. Allowing dispatching", dispatcherName);
+ return false;
+ }
+ DrainingHashEntry entry = drainingHashes.get(stickyKeyHash);
+ // if the entry is not found, the hash is not draining. Don't block the hash.
+ if (entry == null) {
+ return false;
+ }
+ // hash has been reassigned to the original consumer, remove the entry
+ // and don't block the hash
+ if (entry.getConsumer() == consumer) {
+ log.info("[{}] Hash {} has been reassigned consumer {}. "
+ + "The draining hash entry with refCount={} will be removed.",
+ dispatcherName, stickyKeyHash, entry.getConsumer(), entry.refCount);
+ drainingHashes.remove(stickyKeyHash, entry);
+ return false;
+ }
+ // increment the blocked count which is used to determine if the hash is blocking
+ // dispatching to other consumers
+ entry.incrementBlockedCount();
+ // block the hash
+ return true;
+ }
+
+ /**
+ * Get the entry for a given sticky key hash.
+ *
+ * @param stickyKeyHash the sticky key hash
+ * @return the draining hash entry, or null if not found
+ */
+ public synchronized DrainingHashEntry getEntry(int stickyKeyHash) {
+ return stickyKeyHash != 0 ? drainingHashes.get(stickyKeyHash) : null;
+ }
+
+ /**
+ * Clear all entries in the draining hashes tracker.
+ */
+ public synchronized void clear() {
+ drainingHashes.clear();
+ }
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java
index efa89a8ff16f6..e4870bf251ecb 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java
@@ -20,7 +20,7 @@
import com.google.common.annotations.VisibleForTesting;
import io.netty.buffer.ByteBuf;
-import java.nio.charset.StandardCharsets;
+import java.util.function.ToIntFunction;
import javax.annotation.Nullable;
import lombok.Getter;
import org.apache.bookkeeper.mledger.Entry;
@@ -29,11 +29,12 @@
import org.apache.pulsar.common.protocol.Commands;
public class EntryAndMetadata implements Entry {
-
+ private static final int STICKY_KEY_HASH_NOT_INITIALIZED = -1;
private final Entry entry;
@Getter
@Nullable
private final MessageMetadata metadata;
+ int stickyKeyHash = STICKY_KEY_HASH_NOT_INITIALIZED;
private EntryAndMetadata(final Entry entry, @Nullable final MessageMetadata metadata) {
this.entry = entry;
@@ -45,22 +46,15 @@ public static EntryAndMetadata create(final Entry entry, final MessageMetadata m
}
@VisibleForTesting
- static EntryAndMetadata create(final Entry entry) {
+ public static EntryAndMetadata create(final Entry entry) {
return create(entry, Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), "", -1));
}
public byte[] getStickyKey() {
if (metadata != null) {
- if (metadata.hasOrderingKey()) {
- return metadata.getOrderingKey();
- } else if (metadata.hasPartitionKey()) {
- return metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8);
- } else if (metadata.hasProducerName() && metadata.hasSequenceId()) {
- String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId();
- return fallbackKey.getBytes(StandardCharsets.UTF_8);
- }
+ return Commands.resolveStickyKey(metadata);
}
- return "NONE_KEY".getBytes(StandardCharsets.UTF_8);
+ return Commands.NONE_KEY;
}
@Override
@@ -114,4 +108,32 @@ public long getEntryId() {
public boolean release() {
return entry.release();
}
+
+ /**
+ * Get cached sticky key hash or calculate it based on the sticky key if it's not cached.
+ *
+ * @param makeStickyKeyHash function to calculate the sticky key hash
+ * @return the sticky key hash
+ */
+ public int getOrUpdateCachedStickyKeyHash(ToIntFunction makeStickyKeyHash) {
+ if (stickyKeyHash == STICKY_KEY_HASH_NOT_INITIALIZED) {
+ stickyKeyHash = makeStickyKeyHash.applyAsInt(getStickyKey());
+ }
+ return stickyKeyHash;
+ }
+
+ /**
+ * Get cached sticky key hash or return STICKY_KEY_HASH_NOT_SET if it's not cached.
+ *
+ * @return the cached sticky key hash or STICKY_KEY_HASH_NOT_SET if it's not cached
+ */
+ public int getCachedStickyKeyHash() {
+ return stickyKeyHash != STICKY_KEY_HASH_NOT_INITIALIZED ? stickyKeyHash
+ : StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET;
+ }
+
+ @VisibleForTesting
+ public Entry unwrap() {
+ return entry;
+ }
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java
new file mode 100644
index 0000000000000..80996c395e352
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java
@@ -0,0 +1,26 @@
+/*
+ * 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 org.apache.pulsar.client.api.Range;
+
+/**
+ * Hash range assignment in {@link StickyKeyConsumerSelector} implementations.
+ */
+public record HashRangeAssignment(Range range, Consumer consumer) {}
\ No newline at end of file
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java
index a9fea5b39bf82..b90aef739f2b1 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java
@@ -27,7 +27,6 @@
import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException;
import org.apache.pulsar.client.api.Range;
-import org.apache.pulsar.common.util.FutureUtil;
/**
* This is a consumer selector based fixed hash range.
@@ -56,11 +55,11 @@
*
*/
public class HashRangeAutoSplitStickyKeyConsumerSelector implements StickyKeyConsumerSelector {
-
private final int rangeSize;
-
+ private final Range keyHashRange;
private final ConcurrentSkipListMap rangeMap;
private final Map consumerRange;
+ private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot;
public HashRangeAutoSplitStickyKeyConsumerSelector() {
this(DEFAULT_RANGE_SIZE);
@@ -76,10 +75,12 @@ public HashRangeAutoSplitStickyKeyConsumerSelector(int rangeSize) {
this.rangeMap = new ConcurrentSkipListMap<>();
this.consumerRange = new HashMap<>();
this.rangeSize = rangeSize;
+ this.keyHashRange = Range.of(0, rangeSize - 1);
+ this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty();
}
@Override
- public synchronized CompletableFuture addConsumer(Consumer consumer) {
+ public synchronized CompletableFuture addConsumer(Consumer consumer) {
if (rangeMap.isEmpty()) {
rangeMap.put(rangeSize, consumer);
consumerRange.put(consumer, rangeSize);
@@ -87,14 +88,18 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) {
try {
splitRange(findBiggestRange(), consumer);
} catch (ConsumerAssignException e) {
- return FutureUtil.failedFuture(e);
+ return CompletableFuture.failedFuture(e);
}
}
- return CompletableFuture.completedFuture(null);
+ ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot();
+ ImpactedConsumersResult impactedConsumers =
+ consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter);
+ consumerHashAssignmentsSnapshot = assignmentsAfter;
+ return CompletableFuture.completedFuture(impactedConsumers);
}
@Override
- public synchronized void removeConsumer(Consumer consumer) {
+ public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) {
Integer removeRange = consumerRange.remove(consumer);
if (removeRange != null) {
if (removeRange == rangeSize && rangeMap.size() > 1) {
@@ -106,28 +111,40 @@ public synchronized void removeConsumer(Consumer consumer) {
rangeMap.remove(removeRange);
}
}
+ ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot();
+ ImpactedConsumersResult impactedConsumers =
+ consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter);
+ consumerHashAssignmentsSnapshot = assignmentsAfter;
+ return impactedConsumers;
}
@Override
public Consumer select(int hash) {
if (!rangeMap.isEmpty()) {
- int slot = hash % rangeSize;
- return rangeMap.ceilingEntry(slot).getValue();
+ return rangeMap.ceilingEntry(hash).getValue();
} else {
return null;
}
}
@Override
- public Map> getConsumerKeyHashRanges() {
- Map> result = new HashMap<>();
+ public Range getKeyHashRange() {
+ return keyHashRange;
+ }
+
+ @Override
+ public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() {
+ return consumerHashAssignmentsSnapshot;
+ }
+
+ private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() {
+ List result = new ArrayList<>();
int start = 0;
- for (Map.Entry entry: rangeMap.entrySet()) {
- result.computeIfAbsent(entry.getValue(), key -> new ArrayList<>())
- .add(Range.of(start, entry.getKey()));
+ for (Entry entry: rangeMap.entrySet()) {
+ result.add(new HashRangeAssignment(Range.of(start, entry.getKey()), entry.getValue()));
start = entry.getKey() + 1;
}
- return result;
+ return ConsumerHashAssignmentsSnapshot.of(result);
}
private int findBiggestRange() {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java
index 78bad1b2c400e..7c76d9dca7456 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java
@@ -20,7 +20,6 @@
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
@@ -36,9 +35,10 @@
* else there'll be chance that a key fall in a `whole` that not handled by any consumer.
*/
public class HashRangeExclusiveStickyKeyConsumerSelector implements StickyKeyConsumerSelector {
-
private final int rangeSize;
+ private final Range keyHashRange;
private final ConcurrentSkipListMap rangeMap;
+ private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot;
public HashRangeExclusiveStickyKeyConsumerSelector() {
this(DEFAULT_RANGE_SIZE);
@@ -50,21 +50,23 @@ public HashRangeExclusiveStickyKeyConsumerSelector(int rangeSize) {
throw new IllegalArgumentException("range size must greater than 0");
}
this.rangeSize = rangeSize;
+ this.keyHashRange = Range.of(0, rangeSize - 1);
this.rangeMap = new ConcurrentSkipListMap<>();
+ this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty();
}
@Override
- public synchronized CompletableFuture addConsumer(Consumer consumer) {
- return validateKeySharedMeta(consumer).thenRun(() -> {
+ public synchronized CompletableFuture addConsumer(Consumer consumer) {
+ return validateKeySharedMeta(consumer).thenApply(__ -> {
try {
- internalAddConsumer(consumer);
+ return internalAddConsumer(consumer);
} catch (BrokerServiceException.ConsumerAssignException e) {
throw FutureUtil.wrapToCompletionException(e);
}
});
}
- private synchronized void internalAddConsumer(Consumer consumer)
+ private synchronized ImpactedConsumersResult internalAddConsumer(Consumer consumer)
throws BrokerServiceException.ConsumerAssignException {
Consumer conflictingConsumer = findConflictingConsumer(consumer.getKeySharedMeta().getHashRangesList());
if (conflictingConsumer != null) {
@@ -75,37 +77,49 @@ private synchronized void internalAddConsumer(Consumer consumer)
rangeMap.put(intRange.getStart(), consumer);
rangeMap.put(intRange.getEnd(), consumer);
}
+ ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot();
+ ImpactedConsumersResult impactedConsumers =
+ consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter);
+ consumerHashAssignmentsSnapshot = assignmentsAfter;
+ return impactedConsumers;
}
@Override
- public void removeConsumer(Consumer consumer) {
+ public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) {
rangeMap.entrySet().removeIf(entry -> entry.getValue().equals(consumer));
+ ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot();
+ ImpactedConsumersResult impactedConsumers =
+ consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter);
+ consumerHashAssignmentsSnapshot = assignmentsAfter;
+ return impactedConsumers;
}
@Override
- public Map> getConsumerKeyHashRanges() {
- Map> result = new HashMap<>();
+ public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() {
+ return consumerHashAssignmentsSnapshot;
+ }
+
+ private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() {
+ List result = new ArrayList<>();
Map.Entry prev = null;
for (Map.Entry entry: rangeMap.entrySet()) {
if (prev == null) {
prev = entry;
} else {
if (prev.getValue().equals(entry.getValue())) {
- result.computeIfAbsent(entry.getValue(), key -> new ArrayList<>())
- .add(Range.of(prev.getKey(), entry.getKey()));
+ result.add(new HashRangeAssignment(Range.of(prev.getKey(), entry.getKey()), entry.getValue()));
}
prev = null;
}
}
- return result;
+ return ConsumerHashAssignmentsSnapshot.of(result);
}
@Override
public Consumer select(int hash) {
if (rangeMap.size() > 0) {
- int slot = hash % rangeSize;
- Map.Entry ceilingEntry = rangeMap.ceilingEntry(slot);
- Map.Entry floorEntry = rangeMap.floorEntry(slot);
+ Map.Entry ceilingEntry = rangeMap.ceilingEntry(hash);
+ Map.Entry floorEntry = rangeMap.floorEntry(hash);
Consumer ceilingConsumer = ceilingEntry != null ? ceilingEntry.getValue() : null;
Consumer floorConsumer = floorEntry != null ? floorEntry.getValue() : null;
if (floorConsumer != null && floorConsumer.equals(ceilingConsumer)) {
@@ -173,4 +187,8 @@ Map getRangeConsumer() {
return Collections.unmodifiableMap(rangeMap);
}
+ @Override
+ public Range getKeyHashRange() {
+ return keyHashRange;
+ }
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java
new file mode 100644
index 0000000000000..a525b0501d767
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java
@@ -0,0 +1,59 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import java.util.Map;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+
+/**
+ * Represents the consumers that were impacted by a hash range change in a {@link StickyKeyConsumerSelector}
+ * at a point in time.
+ */
+@EqualsAndHashCode
+@ToString
+public class ImpactedConsumersResult {
+ public interface RemovedHashRangesProcessor {
+ void process(Consumer consumer, RemovedHashRanges removedHashRanges);
+ }
+
+ private final Map removedHashRanges;
+
+ private ImpactedConsumersResult(Map removedHashRanges) {
+ this.removedHashRanges = removedHashRanges;
+ }
+
+ public static ImpactedConsumersResult of(Map removedHashRanges) {
+ return new ImpactedConsumersResult(removedHashRanges);
+ }
+
+ public void processRemovedHashRanges(RemovedHashRangesProcessor processor) {
+ removedHashRanges.forEach((c, r) -> processor.process(c, r));
+ }
+
+ public boolean isEmpty() {
+ return removedHashRanges.isEmpty();
+ }
+
+ @VisibleForTesting
+ Map getRemovedHashRanges() {
+ return removedHashRanges;
+ }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java
new file mode 100644
index 0000000000000..7a728a037dc62
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java
@@ -0,0 +1,424 @@
+/*
+ * 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 it.unimi.dsi.fastutil.ints.IntIntPair;
+import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
+import it.unimi.dsi.fastutil.longs.Long2ObjectRBTreeMap;
+import it.unimi.dsi.fastutil.longs.Long2ObjectSortedMap;
+import it.unimi.dsi.fastutil.objects.ObjectBidirectionalIterator;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+
+/**
+ * A thread-safe map to store pending acks in the consumer.
+ *
+ * The locking solution is used for the draining hashes solution
+ * to ensure that there's a consistent view of the pending acks. This is needed in the DrainingHashesTracker
+ * to ensure that the reference counts are consistent at all times.
+ * Calling forEachAndClose will ensure that no more entries can be added,
+ * therefore no other thread cannot send out entries while the forEachAndClose is being called.
+ * remove is also locked to ensure that there aren't races in the removal of entries while forEachAndClose is
+ * running.
+ */
+public class PendingAcksMap {
+ /**
+ * Callback interface for handling the addition of pending acknowledgments.
+ */
+ public interface PendingAcksAddHandler {
+ /**
+ * Handle the addition of a pending acknowledgment.
+ *
+ * @param consumer the consumer
+ * @param ledgerId the ledger ID
+ * @param entryId the entry ID
+ * @param stickyKeyHash the sticky key hash
+ * @return true if the addition is allowed, false otherwise
+ */
+ boolean handleAdding(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash);
+ }
+
+ /**
+ * Callback interface for handling the removal of pending acknowledgments.
+ */
+ public interface PendingAcksRemoveHandler {
+ /**
+ * Handle the removal of a pending acknowledgment.
+ *
+ * @param consumer the consumer
+ * @param ledgerId the ledger ID
+ * @param entryId the entry ID
+ * @param stickyKeyHash the sticky key hash
+ * @param closing true if the pending ack is being removed because the map is being closed, false
+ * otherwise
+ */
+ void handleRemoving(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash, boolean closing);
+ /**
+ * Start a batch of pending acknowledgment removals.
+ */
+ void startBatch();
+ /**
+ * End a batch of pending acknowledgment removals.
+ */
+ void endBatch();
+ }
+
+ /**
+ * Callback interface for processing pending acknowledgments.
+ */
+ public interface PendingAcksConsumer {
+ /**
+ * Accept a pending acknowledgment.
+ *
+ * @param ledgerId the ledger ID
+ * @param entryId the entry ID
+ * @param batchSize the batch size
+ * @param stickyKeyHash the sticky key hash
+ */
+ void accept(long ledgerId, long entryId, int batchSize, int stickyKeyHash);
+ }
+
+ private final Consumer consumer;
+ private final Long2ObjectSortedMap> pendingAcks;
+ private final Supplier pendingAcksAddHandlerSupplier;
+ private final Supplier pendingAcksRemoveHandlerSupplier;
+ private final Lock readLock;
+ private final Lock writeLock;
+ private boolean closed = false;
+
+ PendingAcksMap(Consumer consumer, Supplier pendingAcksAddHandlerSupplier,
+ Supplier pendingAcksRemoveHandlerSupplier) {
+ this.consumer = consumer;
+ this.pendingAcks = new Long2ObjectRBTreeMap<>();
+ this.pendingAcksAddHandlerSupplier = pendingAcksAddHandlerSupplier;
+ this.pendingAcksRemoveHandlerSupplier = pendingAcksRemoveHandlerSupplier;
+ ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+ this.writeLock = readWriteLock.writeLock();
+ this.readLock = readWriteLock.readLock();
+ }
+
+ /**
+ * Add a pending ack to the map if it's allowed to send a message with the given sticky key hash.
+ * If this method returns false, it means that the pending ack was not added, and it's not allowed to send a
+ * message. In that case, the caller should not send a message and skip the entry.
+ * The sending could be disallowed if the sticky key hash is blocked in the Key_Shared subscription.
+ *
+ * @param ledgerId the ledger ID
+ * @param entryId the entry ID
+ * @param batchSize the batch size
+ * @param stickyKeyHash the sticky key hash
+ * @return true if the pending ack was added, and it's allowed to send a message, false otherwise
+ */
+ public boolean addPendingAckIfAllowed(long ledgerId, long entryId, int batchSize, int stickyKeyHash) {
+ try {
+ writeLock.lock();
+ // prevent adding sticky hash to pending acks if the PendingAcksMap has already been closed
+ // and there's a race condition between closing the consumer and sending new messages
+ if (closed) {
+ return false;
+ }
+ // prevent adding sticky hash to pending acks if it's already in draining hashes
+ // to avoid any race conditions that would break consistency
+ PendingAcksAddHandler pendingAcksAddHandler = pendingAcksAddHandlerSupplier.get();
+ if (pendingAcksAddHandler != null
+ && !pendingAcksAddHandler.handleAdding(consumer, ledgerId, entryId, stickyKeyHash)) {
+ return false;
+ }
+ Long2ObjectSortedMap ledgerPendingAcks =
+ pendingAcks.computeIfAbsent(ledgerId, k -> new Long2ObjectRBTreeMap<>());
+ ledgerPendingAcks.put(entryId, IntIntPair.of(batchSize, stickyKeyHash));
+ return true;
+ } finally {
+ writeLock.unlock();
+ }
+ }
+
+ /**
+ * Get the size of the pending acks map.
+ *
+ * @return the size of the pending acks map
+ */
+ public long size() {
+ try {
+ readLock.lock();
+ return pendingAcks.values().stream().mapToInt(Long2ObjectSortedMap::size).sum();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ /**
+ * Iterate over all the pending acks and process them using the given processor.
+ *
+ * @param processor the processor to handle each pending ack
+ */
+ public void forEach(PendingAcksConsumer processor) {
+ try {
+ readLock.lock();
+ processPendingAcks(processor);
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ // iterate all pending acks and process them
+ private void processPendingAcks(PendingAcksConsumer processor) {
+ // this code uses for loops intentionally, don't refactor to use forEach
+ // iterate the outer map
+ for (Map.Entry> entry : pendingAcks.entrySet()) {
+ Long ledgerId = entry.getKey();
+ Long2ObjectSortedMap ledgerPendingAcks = entry.getValue();
+ // iterate the inner map
+ for (Map.Entry e : ledgerPendingAcks.entrySet()) {
+ Long entryId = e.getKey();
+ IntIntPair batchSizeAndStickyKeyHash = e.getValue();
+ processor.accept(ledgerId, entryId, batchSizeAndStickyKeyHash.leftInt(),
+ batchSizeAndStickyKeyHash.rightInt());
+ }
+ }
+ }
+
+ /**
+ * Iterate over all the pending acks and close the map so that no more entries can be added.
+ * All entries are removed.
+ *
+ * @param processor the processor to handle each pending ack
+ */
+ public void forEachAndClose(PendingAcksConsumer processor) {
+ try {
+ writeLock.lock();
+ closed = true;
+ PendingAcksRemoveHandler pendingAcksRemoveHandler = pendingAcksRemoveHandlerSupplier.get();
+ if (pendingAcksRemoveHandler != null) {
+ try {
+ pendingAcksRemoveHandler.startBatch();
+ processPendingAcks((ledgerId, entryId, batchSize, stickyKeyHash) -> {
+ processor.accept(ledgerId, entryId, batchSize, stickyKeyHash);
+ pendingAcksRemoveHandler.handleRemoving(consumer, ledgerId, entryId, stickyKeyHash, closed);
+ });
+ } finally {
+ pendingAcksRemoveHandler.endBatch();
+ }
+ } else {
+ processPendingAcks(processor);
+ }
+ pendingAcks.clear();
+ } finally {
+ writeLock.unlock();
+ }
+ }
+
+ /**
+ * Check if the map contains a pending ack for the given ledger ID and entry ID.
+ *
+ * @param ledgerId the ledger ID
+ * @param entryId the entry ID
+ * @return true if the map contains the pending ack, false otherwise
+ */
+ public boolean contains(long ledgerId, long entryId) {
+ try {
+ readLock.lock();
+ Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId);
+ if (ledgerMap == null) {
+ return false;
+ }
+ return ledgerMap.containsKey(entryId);
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ /**
+ * Get the pending ack for the given ledger ID and entry ID.
+ *
+ * @param ledgerId the ledger ID
+ * @param entryId the entry ID
+ * @return the pending ack, or null if not found
+ */
+ public IntIntPair get(long ledgerId, long entryId) {
+ try {
+ readLock.lock();
+ Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId);
+ if (ledgerMap == null) {
+ return null;
+ }
+ return ledgerMap.get(entryId);
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ /**
+ * Remove the pending ack for the given ledger ID, entry ID, batch size, and sticky key hash.
+ *
+ * @param ledgerId the ledger ID
+ * @param entryId the entry ID
+ * @param batchSize the batch size
+ * @param stickyKeyHash the sticky key hash
+ * @return true if the pending ack was removed, false otherwise
+ */
+ public boolean remove(long ledgerId, long entryId, int batchSize, int stickyKeyHash) {
+ try {
+ writeLock.lock();
+ Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId);
+ if (ledgerMap == null) {
+ return false;
+ }
+ boolean removed = ledgerMap.remove(entryId, IntIntPair.of(batchSize, stickyKeyHash));
+ if (removed) {
+ handleRemovePendingAck(ledgerId, entryId, stickyKeyHash);
+ }
+ if (removed && ledgerMap.isEmpty()) {
+ pendingAcks.remove(ledgerId);
+ }
+ return removed;
+ } finally {
+ writeLock.unlock();
+ }
+ }
+
+ /**
+ * Remove the pending ack for the given ledger ID and entry ID.
+ *
+ * @param ledgerId the ledger ID
+ * @param entryId the entry ID
+ * @return true if the pending ack was removed, false otherwise
+ */
+ public boolean remove(long ledgerId, long entryId) {
+ try {
+ writeLock.lock();
+ Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId);
+ if (ledgerMap == null) {
+ return false;
+ }
+ IntIntPair removedEntry = ledgerMap.remove(entryId);
+ boolean removed = removedEntry != null;
+ if (removed) {
+ int stickyKeyHash = removedEntry.rightInt();
+ handleRemovePendingAck(ledgerId, entryId, stickyKeyHash);
+ }
+ if (removed && ledgerMap.isEmpty()) {
+ pendingAcks.remove(ledgerId);
+ }
+ return removed;
+ } finally {
+ writeLock.unlock();
+ }
+ }
+
+ /**
+ * Remove all pending acks up to the given ledger ID and entry ID.
+ *
+ * @param markDeleteLedgerId the ledger ID up to which to remove pending acks
+ * @param markDeleteEntryId the entry ID up to which to remove pending acks
+ */
+ public void removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) {
+ internalRemoveAllUpTo(markDeleteLedgerId, markDeleteEntryId, false);
+ }
+
+ /**
+ * Removes all pending acknowledgments up to the specified ledger ID and entry ID.
+ *
+ * ReadWriteLock doesn't support upgrading from read lock to write lock.
+ * This method first checks if there's anything to remove using a read lock and if there is, exits
+ * and retries with a write lock to make the removals.
+ *
+ * @param markDeleteLedgerId the ledger ID up to which to remove pending acks
+ * @param markDeleteEntryId the entry ID up to which to remove pending acks
+ * @param useWriteLock true if the method should use a write lock, false otherwise
+ */
+ private void internalRemoveAllUpTo(long markDeleteLedgerId, long markDeleteEntryId, boolean useWriteLock) {
+ PendingAcksRemoveHandler pendingAcksRemoveHandler = pendingAcksRemoveHandlerSupplier.get();
+ // track if the write lock was acquired
+ boolean acquiredWriteLock = false;
+ // track if a batch was started
+ boolean batchStarted = false;
+ // track if the method should retry with a write lock
+ boolean retryWithWriteLock = false;
+ try {
+ if (useWriteLock) {
+ writeLock.lock();
+ acquiredWriteLock = true;
+ } else {
+ readLock.lock();
+ }
+ ObjectBidirectionalIterator>> ledgerMapIterator =
+ pendingAcks.headMap(markDeleteLedgerId + 1).long2ObjectEntrySet().iterator();
+ while (ledgerMapIterator.hasNext()) {
+ Long2ObjectMap.Entry> entry = ledgerMapIterator.next();
+ long ledgerId = entry.getLongKey();
+ Long2ObjectSortedMap ledgerMap = entry.getValue();
+ Long2ObjectSortedMap ledgerMapHead;
+ if (ledgerId == markDeleteLedgerId) {
+ ledgerMapHead = ledgerMap.headMap(markDeleteEntryId + 1);
+ } else {
+ ledgerMapHead = ledgerMap;
+ }
+ ObjectBidirectionalIterator> entryMapIterator =
+ ledgerMapHead.long2ObjectEntrySet().iterator();
+ while (entryMapIterator.hasNext()) {
+ Long2ObjectMap.Entry intIntPairEntry = entryMapIterator.next();
+ long entryId = intIntPairEntry.getLongKey();
+ if (!acquiredWriteLock) {
+ retryWithWriteLock = true;
+ return;
+ }
+ if (pendingAcksRemoveHandler != null) {
+ if (!batchStarted) {
+ pendingAcksRemoveHandler.startBatch();
+ batchStarted = true;
+ }
+ int stickyKeyHash = intIntPairEntry.getValue().rightInt();
+ pendingAcksRemoveHandler.handleRemoving(consumer, ledgerId, entryId, stickyKeyHash, closed);
+ }
+ entryMapIterator.remove();
+ }
+ if (ledgerMap.isEmpty()) {
+ if (!acquiredWriteLock) {
+ retryWithWriteLock = true;
+ return;
+ }
+ ledgerMapIterator.remove();
+ }
+ }
+ } finally {
+ if (batchStarted) {
+ pendingAcksRemoveHandler.endBatch();
+ }
+ if (acquiredWriteLock) {
+ writeLock.unlock();
+ } else {
+ readLock.unlock();
+ if (retryWithWriteLock) {
+ internalRemoveAllUpTo(markDeleteLedgerId, markDeleteEntryId, true);
+ }
+ }
+ }
+ }
+
+ private void handleRemovePendingAck(long ledgerId, long entryId, int stickyKeyHash) {
+ PendingAcksRemoveHandler pendingAcksRemoveHandler = pendingAcksRemoveHandlerSupplier.get();
+ if (pendingAcksRemoveHandler != null) {
+ pendingAcksRemoveHandler.handleRemoving(consumer, ledgerId, entryId, stickyKeyHash, closed);
+ }
+ }
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java
index b4578711027ef..c39b722888f71 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java
@@ -186,7 +186,7 @@ public boolean isSuccessorTo(Producer other) {
&& other.getEpoch() < epoch;
}
- public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndPayload, long batchSize,
+ public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndPayload, int batchSize,
boolean isChunked, boolean isMarker, Position position) {
if (checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize, position)) {
publishMessageToTopic(headersAndPayload, sequenceId, batchSize, isChunked, isMarker, position);
@@ -194,7 +194,7 @@ public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndP
}
public void publishMessage(long producerId, long lowestSequenceId, long highestSequenceId,
- ByteBuf headersAndPayload, long batchSize, boolean isChunked, boolean isMarker, Position position) {
+ ByteBuf headersAndPayload, int batchSize, boolean isChunked, boolean isMarker, Position position) {
if (lowestSequenceId > highestSequenceId) {
cnx.execute(() -> {
cnx.getCommandSender().sendSendError(producerId, highestSequenceId, ServerError.MetadataError,
@@ -209,7 +209,7 @@ public void publishMessage(long producerId, long lowestSequenceId, long highestS
}
}
- public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf headersAndPayload, long batchSize,
+ public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf headersAndPayload, int batchSize,
Position position) {
if (!isShadowTopic && position != null) {
cnx.execute(() -> {
@@ -267,7 +267,7 @@ public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf he
return true;
}
- private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, long batchSize, boolean isChunked,
+ private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, int batchSize, boolean isChunked,
boolean isMarker, Position position) {
MessagePublishContext messagePublishContext =
MessagePublishContext.get(this, sequenceId, headersAndPayload.readableBytes(),
@@ -280,7 +280,7 @@ private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, l
}
private void publishMessageToTopic(ByteBuf headersAndPayload, long lowestSequenceId, long highestSequenceId,
- long batchSize, boolean isChunked, boolean isMarker, Position position) {
+ int batchSize, boolean isChunked, boolean isMarker, Position position) {
MessagePublishContext messagePublishContext = MessagePublishContext.get(this, lowestSequenceId,
highestSequenceId, headersAndPayload.readableBytes(), batchSize,
isChunked, System.nanoTime(), isMarker, position);
@@ -375,7 +375,7 @@ private static final class MessagePublishContext implements PublishContext, Runn
private long ledgerId;
private long entryId;
private int msgSize;
- private long batchSize;
+ private int batchSize;
private boolean chunked;
private boolean isMarker;
@@ -551,7 +551,7 @@ public void run() {
recycle();
}
- static MessagePublishContext get(Producer producer, long sequenceId, int msgSize, long batchSize,
+ static MessagePublishContext get(Producer producer, long sequenceId, int msgSize, int batchSize,
boolean chunked, long startTimeNs, boolean isMarker, Position position) {
MessagePublishContext callback = RECYCLER.get();
callback.producer = producer;
@@ -572,7 +572,7 @@ static MessagePublishContext get(Producer producer, long sequenceId, int msgSize
}
static MessagePublishContext get(Producer producer, long lowestSequenceId, long highestSequenceId, int msgSize,
- long batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) {
+ int batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) {
MessagePublishContext callback = RECYCLER.get();
callback.producer = producer;
callback.sequenceId = lowestSequenceId;
@@ -628,7 +628,7 @@ public void recycle() {
msgSize = 0;
ledgerId = -1L;
entryId = -1L;
- batchSize = 0L;
+ batchSize = 0;
startTimeNs = -1L;
chunked = false;
isMarker = false;
@@ -795,7 +795,7 @@ public void checkEncryption() {
}
public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, long highSequenceId,
- ByteBuf headersAndPayload, long batchSize, boolean isChunked, boolean isMarker) {
+ ByteBuf headersAndPayload, int batchSize, boolean isChunked, boolean isMarker) {
if (!checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize, null)) {
return;
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java
new file mode 100644
index 0000000000000..1833c243f8955
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java
@@ -0,0 +1,74 @@
+/*
+ * 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.List;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+import org.apache.pulsar.client.api.Range;
+
+/**
+ * Represents the hash ranges which were removed from an existing consumer by a change in the hash range assignments.
+ */
+@EqualsAndHashCode
+@ToString
+public class RemovedHashRanges {
+ private final Range[] sortedRanges;
+
+ private RemovedHashRanges(List ranges) {
+ // Converts the set of ranges to an array to avoid iterator allocation
+ // when the ranges are iterator multiple times in the pending acknowledgments loop.
+ this.sortedRanges = ranges.toArray(new Range[0]);
+ validateSortedRanges();
+ }
+
+ private void validateSortedRanges() {
+ for (int i = 0; i < sortedRanges.length - 1; i++) {
+ if (sortedRanges[i].getStart() >= sortedRanges[i + 1].getStart()) {
+ throw new IllegalArgumentException(
+ "Ranges must be sorted: " + sortedRanges[i] + " and " + sortedRanges[i + 1]);
+ }
+ if (sortedRanges[i].getEnd() >= sortedRanges[i + 1].getStart()) {
+ throw new IllegalArgumentException(
+ "Ranges must not overlap: " + sortedRanges[i] + " and " + sortedRanges[i + 1]);
+ }
+ }
+ }
+
+ public static RemovedHashRanges of(List ranges) {
+ return new RemovedHashRanges(ranges);
+ }
+
+ /**
+ * Checks if the sticky key hash is contained in the impacted hash ranges.
+ */
+ public boolean containsStickyKey(int stickyKeyHash) {
+ for (Range range : sortedRanges) {
+ if (range.contains(stickyKeyHash)) {
+ return true;
+ }
+ // Since ranges are sorted, stop checking further ranges if the start of the current range is
+ // greater than the stickyKeyHash.
+ if (range.getStart() > stickyKeyHash) {
+ return false;
+ }
+ }
+ return false;
+ }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java
index e0ed75020bc82..1ead3f946c24d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java
@@ -22,50 +22,94 @@
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import org.apache.pulsar.client.api.Range;
-import org.apache.pulsar.common.util.Murmur3_32Hash;
+/**
+ * Abstraction for selecting the same consumer based on a key.
+ * This interface provides methods to add and remove consumers,
+ * select a consumer based on a sticky key or hash, and retrieve
+ * the hash range assignments for consumers. This is used by the Key_Shared implementation.
+ */
public interface StickyKeyConsumerSelector {
-
+ /**
+ * The default range size used for hashing.
+ * This should be a power of 2 so that it's compatible with all implementations.
+ */
int DEFAULT_RANGE_SIZE = 2 << 15;
+ /**
+ * The value used to indicate that sticky key hash is not set.
+ * This value cannot be -1 since some of the data structures require non-negative values.
+ */
+ int STICKY_KEY_HASH_NOT_SET = 0;
+
/**
* Add a new consumer.
*
- * @param consumer new consumer
+ * @param consumer the new consumer to be added
+ * @return a CompletableFuture that completes with the result of impacted consumers.
+ * The result contains information about the existing consumers whose hash ranges were affected
+ * by the addition of the new consumer.
*/
- CompletableFuture addConsumer(Consumer consumer);
+ CompletableFuture addConsumer(Consumer consumer);
/**
* Remove the consumer.
- * @param consumer consumer to be removed
+ *
+ * @param consumer the consumer to be removed
+ * @return the result of impacted consumers. The result contains information about the existing consumers
+ * whose hash ranges were affected by the removal of the consumer.
*/
- void removeConsumer(Consumer consumer);
+ ImpactedConsumersResult removeConsumer(Consumer consumer);
/**
* Select a consumer by sticky key.
*
- * @param stickyKey sticky key
- * @return consumer
+ * @param stickyKey the sticky key to select the consumer
+ * @return the selected consumer
*/
default Consumer select(byte[] stickyKey) {
return select(makeStickyKeyHash(stickyKey));
}
- static int makeStickyKeyHash(byte[] stickyKey) {
- return Murmur3_32Hash.getInstance().makeHash(stickyKey);
+ /**
+ * Make a hash from the sticky key. The hash value is in the range returned by the {@link #getKeyHashRange()}
+ * method instead of in the full range of integers. In other words, this returns the "slot".
+ *
+ * @param stickyKey the sticky key to hash
+ * @return the generated hash value
+ */
+ default int makeStickyKeyHash(byte[] stickyKey) {
+ return StickyKeyConsumerSelectorUtils.makeStickyKeyHash(stickyKey, getKeyHashRange());
}
/**
* Select a consumer by hash.
*
- * @param hash hash corresponding to sticky key
- * @return consumer
+ * @param hash the hash corresponding to the sticky key
+ * @return the selected consumer
*/
Consumer select(int hash);
+ /**
+ * Get the full range of hash values used by this selector. The upper bound is exclusive.
+ *
+ * @return the full range of hash values
+ */
+ Range getKeyHashRange();
+
/**
* Get key hash ranges handled by each consumer.
- * @return A map where key is a consumer name and value is list of hash range it receiving message for.
+ *
+ * @return a map where the key is a consumer and the value is a list of hash ranges it is receiving messages for
+ */
+ default Map> getConsumerKeyHashRanges() {
+ return getConsumerHashAssignmentsSnapshot().getRangesByConsumer();
+ }
+
+ /**
+ * Get the current mappings of hash range to consumer.
+ *
+ * @return a snapshot of the consumer hash assignments
*/
- Map> getConsumerKeyHashRanges();
-}
+ ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot();
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java
new file mode 100644
index 0000000000000..03a107422ddbd
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java
@@ -0,0 +1,51 @@
+/*
+ * 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 static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET;
+import org.apache.pulsar.client.api.Range;
+import org.apache.pulsar.common.util.Hash;
+import org.apache.pulsar.common.util.Murmur3_32Hash;
+
+/**
+ * Internal utility class for {@link StickyKeyConsumerSelector} implementations.
+ */
+class StickyKeyConsumerSelectorUtils {
+ private static final Hash HASH_INSTANCE = Murmur3_32Hash.getInstance();
+
+ /**
+ * Generates a sticky key hash from the given sticky key within the specified range.
+ * This method shouldn't be used by other classes than {@link StickyKeyConsumerSelector} implementations.
+ * To create a sticky key hash, use {@link StickyKeyConsumerSelector#makeStickyKeyHash(byte[])} instead which
+ * is an instance method of a {@link StickyKeyConsumerSelector}.
+ *
+ * @param stickyKey the sticky key to hash
+ * @param fullHashRange hash range to generate the hash value within
+ * @return the generated hash value, ensuring it is not zero (since zero is a special value in dispatchers)
+ */
+ static int makeStickyKeyHash(byte[] stickyKey, Range fullHashRange) {
+ int hashValue = HASH_INSTANCE.makeHash(stickyKey) % fullHashRange.size() + fullHashRange.getStart();
+ // Avoid using STICKY_KEY_HASH_NOT_SET as hash value
+ if (hashValue == STICKY_KEY_HASH_NOT_SET) {
+ // use next value as hash value
+ hashValue = STICKY_KEY_HASH_NOT_SET + 1;
+ }
+ return hashValue;
+ }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java
index fb7bd22de94a7..ce674cf471ef0 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java
@@ -101,10 +101,10 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) {
consumerList.remove(consumer);
}
throw FutureUtil.wrapToCompletionException(ex);
- } else {
- return value;
}
- }));
+ return value;
+ })).thenAccept(__ -> {
+ });
}
@Override
@@ -152,7 +152,7 @@ public void sendMessages(List entries) {
for (Entry entry : entries) {
byte[] stickyKey = peekStickyKey(entry.getDataBuffer());
- int stickyKeyHash = StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey);
+ int stickyKeyHash = selector.makeStickyKeyHash(stickyKey);
Consumer consumer = selector.select(stickyKeyHash);
if (consumer != null) {
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 fa6e1412151b6..b34a0b454385f 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
@@ -18,7 +18,7 @@
*/
package org.apache.pulsar.broker.service.persistent;
-import com.google.common.collect.ComparisonChain;
+import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
@@ -65,6 +65,9 @@ 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) {
+ throw new IllegalArgumentException("Sticky key hash is not set. It is required.");
+ }
boolean inserted = hashesToBeBlocked.putIfAbsent(ledgerId, entryId, stickyKeyHash, 0);
if (!inserted) {
hashesToBeBlocked.put(ledgerId, entryId, stickyKeyHash, 0);
@@ -108,18 +111,20 @@ public Long getHash(long ledgerId, long entryId) {
}
public void removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) {
- if (!allowOutOfOrderDelivery) {
+ boolean bitsCleared = messagesToRedeliver.removeUpTo(markDeleteLedgerId, markDeleteEntryId + 1);
+ // only if bits have been clear, and we are not allowing out of order delivery, we need to remove the hashes
+ // removing hashes is a relatively expensive operation, so we should only do it when necessary
+ if (bitsCleared && !allowOutOfOrderDelivery) {
List keysToRemove = new ArrayList<>();
hashesToBeBlocked.forEach((ledgerId, entryId, stickyKeyHash, none) -> {
- if (ComparisonChain.start().compare(ledgerId, markDeleteLedgerId).compare(entryId, markDeleteEntryId)
- .result() <= 0) {
+ if (ledgerId < markDeleteLedgerId || (ledgerId == markDeleteLedgerId && entryId <= markDeleteEntryId)) {
keysToRemove.add(new LongPair(ledgerId, entryId));
}
});
- keysToRemove.forEach(longPair -> removeFromHashBlocker(longPair.first, longPair.second));
- keysToRemove.clear();
+ for (LongPair longPair : keysToRemove) {
+ removeFromHashBlocker(longPair.first, longPair.second);
+ }
}
- messagesToRedeliver.removeUpTo(markDeleteLedgerId, markDeleteEntryId + 1);
}
public boolean isEmpty() {
@@ -141,7 +146,7 @@ public String toString() {
public boolean containsStickyKeyHashes(Set stickyKeyHashes) {
if (!allowOutOfOrderDelivery) {
for (Integer stickyKeyHash : stickyKeyHashes) {
- if (hashesRefCount.containsKey(stickyKeyHash)) {
+ if (stickyKeyHash != STICKY_KEY_HASH_NOT_SET && hashesRefCount.containsKey(stickyKeyHash)) {
return true;
}
}
@@ -150,7 +155,8 @@ public boolean containsStickyKeyHashes(Set stickyKeyHashes) {
}
public boolean containsStickyKeyHash(int stickyKeyHash) {
- return !allowOutOfOrderDelivery && hashesRefCount.containsKey(stickyKeyHash);
+ return !allowOutOfOrderDelivery
+ && stickyKeyHash != STICKY_KEY_HASH_NOT_SET && hashesRefCount.containsKey(stickyKeyHash);
}
public Optional getFirstPositionInReplay() {
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 d479d8f384ee9..73d152bab1a60 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
@@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.service.persistent;
+import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET;
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;
@@ -46,6 +47,7 @@
import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.PositionFactory;
+import org.apache.commons.lang3.mutable.MutableBoolean;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory;
@@ -67,7 +69,6 @@
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;
@@ -132,6 +133,7 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul
"blockedDispatcherOnUnackedMsgs");
protected Optional dispatchRateLimiter = Optional.empty();
private AtomicBoolean isRescheduleReadInProgress = new AtomicBoolean(false);
+ private final AtomicBoolean readMoreEntriesAsyncRequested = new AtomicBoolean(false);
protected final ExecutorService dispatchMessagesThread;
private final SharedConsumerAssignor assignor;
// tracks how many entries were processed by consumers in the last trySendMessagesToConsumers call
@@ -144,6 +146,8 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul
protected enum ReadType {
Normal, Replay
}
+ private Position lastMarkDeletePositionBeforeReadMoreEntries;
+ private volatile long readMoreEntriesCallCount;
public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor,
Subscription subscription) {
@@ -164,7 +168,7 @@ public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCurso
: RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED;
this.readBatchSize = serviceConfig.getDispatcherMaxReadBatchSize();
this.initializeDispatchRateLimiterIfNeeded();
- this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addMessageToReplay);
+ this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addEntryToReplay);
ServiceConfiguration serviceConfiguration = topic.getBrokerService().pulsar().getConfiguration();
this.readFailureBackoff = new Backoff(
serviceConfiguration.getDispatcherReadFailureBackoffInitialTimeInMs(),
@@ -239,8 +243,12 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE
if (log.isDebugEnabled()) {
log.debug("[{}] Consumer are left, reading more entries", name);
}
- consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> {
- addMessageToReplay(ledgerId, entryId, stickyKeyHash);
+ MutableBoolean notifyAddedToReplay = new MutableBoolean(false);
+ consumer.getPendingAcks().forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> {
+ boolean addedToReplay = addMessageToReplay(ledgerId, entryId, stickyKeyHash);
+ if (addedToReplay) {
+ notifyAddedToReplay.setTrue();
+ }
});
totalAvailablePermits -= consumer.getAvailablePermits();
if (log.isDebugEnabled()) {
@@ -248,7 +256,9 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE
+ "New dispatcher permit count is {}", name, consumer.getAvailablePermits(),
totalAvailablePermits);
}
- readMoreEntries();
+ if (notifyAddedToReplay.booleanValue()) {
+ notifyRedeliveryMessageAdded();
+ }
}
} else {
/**
@@ -264,7 +274,12 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE
}
}
- private synchronized void clearComponentsAfterRemovedAllConsumers() {
+ protected synchronized void internalRemoveConsumer(Consumer consumer) {
+ consumerSet.removeAll(consumer);
+ consumerList.remove(consumer);
+ }
+
+ protected synchronized void clearComponentsAfterRemovedAllConsumers() {
cancelPendingRead();
redeliveryMessages.clear();
@@ -298,7 +313,7 @@ private synchronized void internalConsumerFlow(Consumer consumer, int additional
+ "after adding {} permits", name, consumer,
totalAvailablePermits, additionalNumberOfMessages);
}
- readMoreEntries();
+ readMoreEntriesAsync();
}
/**
@@ -306,7 +321,13 @@ private synchronized void internalConsumerFlow(Consumer consumer, int additional
*
*/
public void readMoreEntriesAsync() {
- topic.getBrokerService().executor().execute(this::readMoreEntries);
+ // deduplication for readMoreEntriesAsync calls
+ if (readMoreEntriesAsyncRequested.compareAndSet(false, true)) {
+ topic.getBrokerService().executor().execute(() -> {
+ readMoreEntriesAsyncRequested.set(false);
+ readMoreEntries();
+ });
+ }
}
public synchronized void readMoreEntries() {
@@ -337,6 +358,20 @@ public synchronized void readMoreEntries() {
return;
}
+ // increment the counter for readMoreEntries calls, to track the number of times readMoreEntries is called
+ readMoreEntriesCallCount++;
+
+ // remove possible expired messages from redelivery tracker and pending acks
+ Position markDeletePosition = cursor.getMarkDeletedPosition();
+ if (lastMarkDeletePositionBeforeReadMoreEntries != markDeletePosition) {
+ redeliveryMessages.removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId());
+ for (Consumer consumer : consumerList) {
+ consumer.getPendingAcks()
+ .removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId());
+ }
+ lastMarkDeletePositionBeforeReadMoreEntries = markDeletePosition;
+ }
+
// totalAvailablePermits may be updated by other threads
int firstAvailableConsumerPermits = getFirstAvailableConsumerPermits();
int currentTotalAvailablePermits = Math.max(totalAvailablePermits, firstAvailableConsumerPermits);
@@ -396,21 +431,8 @@ public synchronized void readMoreEntries() {
updateMinReplayedPosition();
messagesToRead = Math.min(messagesToRead, getMaxEntriesReadLimit());
-
- // 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());
- }
+ cursor.asyncReadEntriesWithSkipOrWait(messagesToRead, bytesToRead, this, ReadType.Normal,
+ topic.getMaxReadPosition(), createReadEntriesSkipConditionForNormalRead());
} else {
if (log.isDebugEnabled()) {
log.debug("[{}] Cannot schedule next read until previous one is done", name);
@@ -423,6 +445,19 @@ public synchronized void readMoreEntries() {
}
}
+ protected Predicate createReadEntriesSkipConditionForNormalRead() {
+ Predicate skipCondition = null;
+ // Filter out and skip read delayed messages exist in DelayedDeliveryTracker
+ if (delayedDeliveryTracker.isPresent()) {
+ final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get();
+ if (deliveryTracker instanceof BucketDelayedDeliveryTracker) {
+ skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker)
+ .containsMessage(position.getLedgerId(), position.getEntryId());
+ }
+ }
+ return skipCondition;
+ }
+
/**
* Sets a hard limit on the number of entries to read from the Managed Ledger.
* Subclasses can override this method to set a different limit.
@@ -447,6 +482,10 @@ protected void handleNormalReadNotAllowed() {
// do nothing
}
+ protected long getReadMoreEntriesCallCount() {
+ return readMoreEntriesCallCount;
+ }
+
/**
* Controls whether replaying entries is currently enabled.
* Subclasses can override this method to temporarily disable replaying entries.
@@ -702,7 +741,7 @@ public final synchronized void readEntriesComplete(List entries, Object c
entries.forEach(Entry::release);
cursor.rewind();
shouldRewindBeforeReadingOrReplaying = false;
- readMoreEntries();
+ readMoreEntriesAsync();
return;
}
@@ -799,8 +838,15 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
boolean hasChunk = false;
for (int i = 0; i < metadataArray.length; i++) {
Entry entry = entries.get(i);
- MessageMetadata metadata = entry instanceof EntryAndMetadata ? ((EntryAndMetadata) entry).getMetadata()
- : Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), subscription.toString(), -1);
+ MessageMetadata metadata;
+ if (entry instanceof EntryAndMetadata) {
+ metadata = ((EntryAndMetadata) entry).getMetadata();
+ } else {
+ metadata = Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), subscription.toString(), -1);
+ // cache the metadata in the entry with EntryAndMetadata for later use to avoid re-parsing the metadata
+ // and to carry the metadata and calculated stickyKeyHash with the entry
+ entries.set(i, EntryAndMetadata.create(entry, metadata));
+ }
if (metadata != null) {
remainingMessages += metadata.getNumMessagesInBatch();
if (!hasChunk && metadata.hasUuid()) {
@@ -901,18 +947,17 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
log.debug("[{}] No consumers found with available permits, storing {} positions for later replay", name,
entries.size() - start);
}
- entries.subList(start, entries.size()).forEach(entry -> {
- addEntryToReplay(entry);
- entry.release();
- });
+ entries.subList(start, entries.size()).forEach(this::addEntryToReplay);
}
return true;
}
- protected void addEntryToReplay(Entry entry) {
+ protected boolean addEntryToReplay(Entry entry) {
long stickyKeyHash = getStickyKeyHash(entry);
- addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash);
+ boolean addedToReplay = addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash);
+ entry.release();
+ return addedToReplay;
}
private boolean sendChunkedMessagesToConsumers(ReadType readType,
@@ -930,6 +975,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType,
long totalEntries = 0;
long totalEntriesProcessed = 0;
final AtomicInteger numConsumers = new AtomicInteger(assignResult.size());
+ boolean notifyAddedToReplay = false;
for (Map.Entry> current : assignResult.entrySet()) {
final Consumer consumer = current.getKey();
final List entryAndMetadataList = current.getValue();
@@ -941,7 +987,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType,
if (messagesForC < entryAndMetadataList.size()) {
for (int i = messagesForC; i < entryAndMetadataList.size(); i++) {
final EntryAndMetadata entry = entryAndMetadataList.get(i);
- addMessageToReplay(entry);
+ notifyAddedToReplay |= addEntryToReplay(entry);
entryAndMetadataList.set(i, null);
}
}
@@ -965,7 +1011,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType,
sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker()
).addListener(future -> {
if (future.isDone() && numConsumers.decrementAndGet() == 0) {
- readMoreEntries();
+ readMoreEntriesAsync();
}
});
@@ -978,7 +1024,8 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType,
lastNumberOfEntriesProcessed = (int) totalEntriesProcessed;
acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent);
- return numConsumers.get() == 0; // trigger a new readMoreEntries() call
+ // trigger a new readMoreEntries() call
+ return numConsumers.get() == 0 || notifyAddedToReplay;
}
@Override
@@ -1111,31 +1158,39 @@ public boolean isConsumerAvailable(Consumer consumer) {
@Override
public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch) {
+ if (log.isDebugEnabled()) {
+ log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer,
+ redeliveryMessages);
+ }
+ MutableBoolean addedToReplay = new MutableBoolean(false);
consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> {
if (addMessageToReplay(ledgerId, entryId, stickyKeyHash)) {
redeliveryTracker.incrementAndGetRedeliveryCount((PositionFactory.create(ledgerId, entryId)));
+ addedToReplay.setTrue();
}
});
- if (log.isDebugEnabled()) {
- log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer,
- redeliveryMessages);
+ if (addedToReplay.booleanValue()) {
+ notifyRedeliveryMessageAdded();
}
- readMoreEntries();
}
@Override
public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) {
+ if (log.isDebugEnabled()) {
+ log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions);
+ }
+ MutableBoolean addedToReplay = new MutableBoolean(false);
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);
+ addedToReplay.setTrue();
}
});
- if (log.isDebugEnabled()) {
- log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions);
+ if (addedToReplay.booleanValue()) {
+ notifyRedeliveryMessageAdded();
}
- readMoreEntries();
}
@Override
@@ -1366,11 +1421,6 @@ public void cursorIsReset() {
}
}
- 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);
@@ -1380,6 +1430,13 @@ protected boolean addMessageToReplay(long ledgerId, long entryId, long stickyKey
}
}
+ /**
+ * Notify the dispatcher that a message has been added to the redelivery list.
+ */
+ private void notifyRedeliveryMessageAdded() {
+ readMoreEntriesAsync();
+ }
+
protected boolean addMessageToReplay(long ledgerId, long entryId) {
if (checkIfMessageIsUnacked(ledgerId, entryId)) {
redeliveryMessages.add(ledgerId, entryId);
@@ -1404,7 +1461,7 @@ public boolean checkAndUnblockIfStuck() {
if (totalAvailablePermits > 0 && !havePendingReplayRead && !havePendingRead
&& cursor.getNumberOfEntriesInBacklog(false) > 0) {
log.warn("{}-{} Dispatcher is stuck and unblocking by issuing reads", topic.getName(), name);
- readMoreEntries();
+ readMoreEntriesAsync();
return true;
}
return false;
@@ -1436,10 +1493,10 @@ public ManagedCursor getCursor() {
}
protected int getStickyKeyHash(Entry entry) {
- return StickyKeyConsumerSelector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer()));
+ // There's no need to calculate the hash for Shared subscription
+ return STICKY_KEY_HASH_NOT_SET;
}
-
public Subscription getSubscription() {
return subscription;
}
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 ecd3f19a14028..a78e4e46c0e5a 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
@@ -18,12 +18,11 @@
*/
package org.apache.pulsar.broker.service.persistent;
+import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET;
import com.google.common.annotations.VisibleForTesting;
import java.util.ArrayList;
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.Optional;
@@ -31,24 +30,24 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Predicate;
-import javax.annotation.Nullable;
+import lombok.Getter;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedCursor;
-import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.bookkeeper.mledger.Position;
-import org.apache.bookkeeper.mledger.PositionFactory;
-import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
-import org.apache.commons.collections4.MapUtils;
import org.apache.commons.lang3.mutable.MutableBoolean;
import org.apache.commons.lang3.mutable.MutableInt;
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.DrainingHashesTracker;
+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.ImpactedConsumersResult;
+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.Subscription;
@@ -56,9 +55,8 @@
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.protocol.Commands;
import org.apache.pulsar.common.util.FutureUtil;
-import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet;
-import org.apache.pulsar.common.util.collections.LongPairRangeSet;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -66,25 +64,14 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi
private final boolean allowOutOfOrderDelivery;
private final StickyKeyConsumerSelector selector;
- private final boolean recentlyJoinedConsumerTrackingRequired;
+ private final boolean drainingHashesRequired;
private boolean skipNextReplayToTriggerLookAhead = false;
private final KeySharedMode keySharedMode;
+ @Getter
+ private final DrainingHashesTracker drainingHashesTracker;
- /**
- * 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;
-
- /**
- * The lastSentPosition and the individuallySentPositions are not thread safe.
- */
- @Nullable
- private Position lastSentPosition;
- private final LongPairRangeSet individuallySentPositions;
- private static final LongPairRangeSet.LongPairConsumer positionRangeConverter = PositionFactory::create;
+ private final RescheduleReadHandler rescheduleReadHandler;
PersistentStickyKeyDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor,
Subscription subscription, ServiceConfiguration conf, KeySharedMeta ksm) {
@@ -93,13 +80,13 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi
this.allowOutOfOrderDelivery = ksm.isAllowOutOfOrderDelivery();
this.keySharedMode = ksm.getKeySharedMode();
// recent joined consumer tracking is required only for AUTO_SPLIT mode when out-of-order delivery is disabled
- this.recentlyJoinedConsumerTrackingRequired =
+ this.drainingHashesRequired =
keySharedMode == KeySharedMode.AUTO_SPLIT && !allowOutOfOrderDelivery;
- this.recentlyJoinedConsumers = recentlyJoinedConsumerTrackingRequired ? new LinkedHashMap<>() : null;
- this.individuallySentPositions =
- recentlyJoinedConsumerTrackingRequired
- ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter)
- : null;
+ this.drainingHashesTracker =
+ drainingHashesRequired ? new DrainingHashesTracker(this.getName(), this::stickyKeyHashUnblocked) : null;
+ this.rescheduleReadHandler = new RescheduleReadHandler(conf::getKeySharedUnblockingIntervalMs,
+ topic.getBrokerService().executor(), this::cancelPendingRead, () -> reScheduleReadInMs(0),
+ () -> havePendingRead, this::getReadMoreEntriesCallCount, () -> !redeliveryMessages.isEmpty());
switch (this.keySharedMode) {
case AUTO_SPLIT:
if (conf.isSubscriptionKeySharedUseConsistentHashing()) {
@@ -109,16 +96,29 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi
selector = new HashRangeAutoSplitStickyKeyConsumerSelector();
}
break;
-
case STICKY:
this.selector = new HashRangeExclusiveStickyKeyConsumerSelector();
break;
-
default:
throw new IllegalArgumentException("Invalid key-shared mode: " + keySharedMode);
}
}
+ private void stickyKeyHashUnblocked(int stickyKeyHash) {
+ if (log.isDebugEnabled()) {
+ if (stickyKeyHash > -1) {
+ log.debug("[{}] Sticky key hash {} is unblocked", getName(), stickyKeyHash);
+ } else {
+ log.debug("[{}] Some sticky key hashes are unblocked", getName());
+ }
+ }
+ reScheduleReadWithKeySharedUnblockingInterval();
+ }
+
+ private void reScheduleReadWithKeySharedUnblockingInterval() {
+ rescheduleReadHandler.rescheduleRead();
+ }
+
@VisibleForTesting
public StickyKeyConsumerSelector getSelector() {
return selector;
@@ -131,32 +131,52 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) {
consumer.disconnect();
return CompletableFuture.completedFuture(null);
}
- return super.addConsumer(consumer).thenCompose(__ ->
- selector.addConsumer(consumer).handle((result, ex) -> {
- if (ex != null) {
- synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) {
- consumerSet.removeAll(consumer);
- consumerList.remove(consumer);
- }
- throw FutureUtil.wrapToCompletionException(ex);
+ return super.addConsumer(consumer).thenCompose(__ -> selector.addConsumer(consumer))
+ .thenAccept(impactedConsumers -> {
+ // TODO: Add some way to prevent changes in between the time the consumer is added and the
+ // time the draining hashes are applied. It might be fine for ConsistentHashingStickyKeyConsumerSelector
+ // since it's not really asynchronous, although it returns a CompletableFuture
+ if (drainingHashesRequired) {
+ consumer.setPendingAcksAddHandler(this::handleAddingPendingAck);
+ consumer.setPendingAcksRemoveHandler(new PendingAcksMap.PendingAcksRemoveHandler() {
+ @Override
+ public void handleRemoving(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash,
+ boolean closing) {
+ drainingHashesTracker.reduceRefCount(consumer, stickyKeyHash, closing);
}
- return result;
- })
- ).thenRun(() -> {
- synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) {
- if (recentlyJoinedConsumerTrackingRequired) {
- final Position lastSentPositionWhenJoining = updateIfNeededAndGetLastSentPosition();
- if (lastSentPositionWhenJoining != null) {
- consumer.setLastSentPositionWhenJoining(lastSentPositionWhenJoining);
- // If this was the 1st consumer, or if all the messages are already acked, then we
- // don't need to do anything special
- if (recentlyJoinedConsumers != null
- && consumerList.size() > 1
- && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) {
- recentlyJoinedConsumers.put(consumer, lastSentPositionWhenJoining);
- }
+
+ @Override
+ public void startBatch() {
+ drainingHashesTracker.startBatch();
}
- }
+
+ @Override
+ public void endBatch() {
+ drainingHashesTracker.endBatch();
+ }
+ });
+ registerDrainingHashes(consumer, impactedConsumers);
+ }
+ }).exceptionally(ex -> {
+ internalRemoveConsumer(consumer);
+ throw FutureUtil.wrapToCompletionException(ex);
+ });
+ }
+
+ private synchronized void registerDrainingHashes(Consumer skipConsumer,
+ ImpactedConsumersResult impactedConsumers) {
+ impactedConsumers.processRemovedHashRanges((c, removedHashRanges) -> {
+ if (c != skipConsumer) {
+ c.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> {
+ if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) {
+ log.warn("[{}] Sticky key hash was missing for {}:{}", getName(), ledgerId, entryId);
+ return;
+ }
+ if (removedHashRanges.containsStickyKey(stickyKeyHash)) {
+ // add the pending ack to the draining hashes tracker if the hash is in the range
+ drainingHashesTracker.addEntry(c, stickyKeyHash);
+ }
+ });
}
});
}
@@ -164,27 +184,21 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) {
@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);
+ ImpactedConsumersResult impactedConsumers = selector.removeConsumer(consumer);
super.removeConsumer(consumer);
- if (recentlyJoinedConsumerTrackingRequired) {
- recentlyJoinedConsumers.remove(consumer);
- if (consumerList.size() == 1) {
- recentlyJoinedConsumers.clear();
- } else if (consumerList.isEmpty()) {
- // The subscription removes consumers if rewind or reset cursor operations are called.
- // The dispatcher must clear lastSentPosition and individuallySentPositions because
- // these operations trigger re-sending messages.
- lastSentPosition = null;
- individuallySentPositions.clear();
- }
- if (removeConsumersFromRecentJoinedConsumers() || !redeliveryMessages.isEmpty()) {
- readMoreEntries();
- }
+ if (drainingHashesRequired) {
+ // register draining hashes for the impacted consumers and ranges, in case a hash switched from one
+ // consumer to another. This will handle the case where a hash gets switched from an existing
+ // consumer to another existing consumer during removal.
+ registerDrainingHashes(consumer, impactedConsumers);
+ }
+ }
+
+ @Override
+ protected synchronized void clearComponentsAfterRemovedAllConsumers() {
+ super.clearComponentsAfterRemovedAllConsumers();
+ if (drainingHashesRequired) {
+ drainingHashesTracker.clear();
}
}
@@ -226,11 +240,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
name, replayPosition, minReplayedPosition, readType);
}
if (readType == ReadType.Normal) {
- entries.forEach(entry -> {
- long stickyKeyHash = getStickyKeyHash(entry);
- addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash);
- entry.release();
- });
+ entries.forEach(this::addEntryToReplay);
} else if (readType == ReadType.Replay) {
entries.forEach(Entry::release);
}
@@ -241,26 +251,6 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
}
}
- if (recentlyJoinedConsumerTrackingRequired) {
- // Update if the markDeletePosition move forward
- updateIfNeededAndGetLastSentPosition();
-
- // Should not access to individualDeletedMessages from outside managed cursor
- // because it doesn't guarantee thread safety.
- if (lastSentPosition == null) {
- if (cursor.getMarkDeletedPosition() != null) {
- lastSentPosition = ((ManagedCursorImpl) cursor)
- .processIndividuallyDeletedMessagesAndGetMarkDeletedPosition(range -> {
- final Position lower = range.lowerEndpoint();
- final Position upper = range.upperEndpoint();
- individuallySentPositions.addOpenClosed(lower.getLedgerId(), lower.getEntryId(),
- upper.getLedgerId(), upper.getEntryId());
- return true;
- });
- }
- }
- }
-
// returns a boolean indicating whether look-ahead could be useful, when there's a consumer
// with available permits, and it's not able to make progress because of blocked hashes.
MutableBoolean triggerLookAhead = new MutableBoolean();
@@ -276,23 +266,11 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
log.debug("[{}] select consumer {} with messages num {}, read type is {}",
name, consumer.consumerName(), entriesForConsumer.size(), readType);
}
- final ManagedLedger managedLedger = cursor.getManagedLedger();
- for (Entry entry : entriesForConsumer) {
- // remove positions first from replay list first : sendMessages recycles entries
- if (readType == ReadType.Replay) {
+ // remove positions first from replay list first : sendMessages recycles entries
+ if (readType == ReadType.Replay) {
+ for (Entry entry : entriesForConsumer) {
redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId());
}
- // Add positions to individuallySentPositions if necessary
- if (recentlyJoinedConsumerTrackingRequired) {
- final Position position = entry.getPosition();
- // Store to individuallySentPositions even if lastSentPosition is null
- if ((lastSentPosition == null || position.compareTo(lastSentPosition) > 0)
- && !individuallySentPositions.contains(position.getLedgerId(), position.getEntryId())) {
- final Position previousPosition = managedLedger.getPreviousPosition(position);
- individuallySentPositions.addOpenClosed(previousPosition.getLedgerId(),
- previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId());
- }
- }
}
SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal();
@@ -306,7 +284,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(),
getRedeliveryTracker()).addListener(future -> {
if (future.isDone() && remainingConsumersToFinishSending.decrementAndGet() == 0) {
- readMoreEntries();
+ readMoreEntriesAsync();
}
});
@@ -316,60 +294,6 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
totalBytesSent += sendMessageInfo.getTotalBytes();
}
- // Update the last sent position and remove ranges from individuallySentPositions if necessary
- if (recentlyJoinedConsumerTrackingRequired && lastSentPosition != null) {
- final ManagedLedger managedLedger = cursor.getManagedLedger();
- com.google.common.collect.Range range = individuallySentPositions.firstRange();
-
- // If the upper bound is before the last sent position, we need to move ahead as these
- // individuallySentPositions are now irrelevant.
- if (range != null && range.upperEndpoint().compareTo(lastSentPosition) <= 0) {
- individuallySentPositions.removeAtMost(lastSentPosition.getLedgerId(),
- lastSentPosition.getEntryId());
- range = individuallySentPositions.firstRange();
- }
-
- if (range != null) {
- // If the lowerBound is ahead of the last sent position,
- // verify if there are any entries in-between.
- if (range.lowerEndpoint().compareTo(lastSentPosition) <= 0 || managedLedger
- .getNumberOfEntries(com.google.common.collect.Range.openClosed(lastSentPosition,
- range.lowerEndpoint())) <= 0) {
- if (log.isDebugEnabled()) {
- log.debug("[{}] Found a position range to last sent: {}", name, range);
- }
- Position newLastSentPosition = range.upperEndpoint();
- Position positionAfterNewLastSent = managedLedger
- .getNextValidPosition(newLastSentPosition);
- // sometime ranges are connected but belongs to different ledgers
- // so, they are placed sequentially
- // eg: (2:10..3:15] can be returned as (2:10..2:15],[3:0..3:15].
- // So, try to iterate over connected range and found the last non-connected range
- // which gives new last sent position.
- final Position lastConfirmedEntrySnapshot = managedLedger.getLastConfirmedEntry();
- if (lastConfirmedEntrySnapshot != null) {
- while (positionAfterNewLastSent.compareTo(lastConfirmedEntrySnapshot) <= 0) {
- if (individuallySentPositions.contains(positionAfterNewLastSent.getLedgerId(),
- positionAfterNewLastSent.getEntryId())) {
- range = individuallySentPositions.rangeContaining(
- positionAfterNewLastSent.getLedgerId(), positionAfterNewLastSent.getEntryId());
- newLastSentPosition = range.upperEndpoint();
- positionAfterNewLastSent = managedLedger.getNextValidPosition(newLastSentPosition);
- // check if next valid position is also deleted and part of the deleted-range
- continue;
- }
- break;
- }
- }
-
- if (lastSentPosition.compareTo(newLastSentPosition) < 0) {
- lastSentPosition = newLastSentPosition;
- }
- individuallySentPositions.removeAtMost(lastSentPosition.getLedgerId(),
- lastSentPosition.getEntryId());
- }
- }
- }
lastNumberOfEntriesProcessed = (int) totalEntriesProcessed;
@@ -386,7 +310,8 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
// therefore would be most likely only increase the distance between read-position and mark-delete position.
skipNextReplayToTriggerLookAhead = true;
// skip backoff delay before reading ahead in the "look ahead" mode to prevent any additional latency
- skipNextBackoff = true;
+ // only skip the delay if there are more entries to read
+ skipNextBackoff = cursor.hasMoreEntries();
return true;
}
@@ -398,6 +323,37 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
return false;
}
+ /**
+ * Check if the sticky hash is already draining or blocked in the replay queue.
+ * If it is, add the message to replay and return false so that the message isn't sent to a consumer.
+ *
+ * @param ledgerId the ledger id of the message
+ * @param entryId the entry id of the message
+ * @param stickyKeyHash the sticky hash of the message
+ * @return true if the message should be added to pending acks and allow sending, false otherwise
+ */
+ private boolean handleAddingPendingAck(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash) {
+ if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) {
+ log.warn("[{}] Sticky key hash is missing for {}:{}", getName(), ledgerId, entryId);
+ throw new IllegalArgumentException("Sticky key hash is missing for " + ledgerId + ":" + entryId);
+ }
+ DrainingHashesTracker.DrainingHashEntry drainingHashEntry = drainingHashesTracker.getEntry(stickyKeyHash);
+ if (drainingHashEntry != null && drainingHashEntry.getConsumer() != consumer) {
+ log.warn("[{}] Another consumer id {} is already draining hash {}. Skipping adding {}:{} to pending acks "
+ + "for consumer {}. Adding the message to replay.",
+ getName(), drainingHashEntry.getConsumer(), stickyKeyHash, ledgerId, entryId, consumer);
+ addMessageToReplay(ledgerId, entryId, stickyKeyHash);
+ // block message from sending
+ return false;
+ }
+ if (log.isDebugEnabled()) {
+ log.debug("[{}] Adding {}:{} to pending acks for consumer {} with sticky key hash {}",
+ getName(), ledgerId, entryId, consumer, stickyKeyHash);
+ }
+ // allow adding the message to pending acks and sending the message to the consumer
+ return true;
+ }
+
private boolean isReplayQueueSizeBelowLimit() {
return redeliveryMessages.size() < getEffectiveLookAheadLimit();
}
@@ -442,16 +398,22 @@ private Map> filterAndGroupEntriesForDispatching(List> entriesGroupedByConsumer = new HashMap<>();
// permits for consumer, permits are for entries/batches
Map permitsForConsumer = new HashMap<>();
- // maxLastSentPosition cache for consumers, used when recently joined consumers exist
- boolean hasRecentlyJoinedConsumers = hasRecentlyJoinedConsumers();
- Map maxLastSentPositionCache = hasRecentlyJoinedConsumers ? new HashMap<>() : null;
boolean lookAheadAllowed = isReplayQueueSizeBelowLimit();
// in normal read mode, keep track of consumers that are blocked by hash, to check if look-ahead could be useful
Set blockedByHashConsumers = lookAheadAllowed && readType == ReadType.Normal ? new HashSet<>() : null;
// in replay read mode, keep track of consumers for entries, used for look-ahead check
Set consumersForEntriesForLookaheadCheck = lookAheadAllowed ? new HashSet<>() : null;
- for (Entry entry : entries) {
+ for (Entry inputEntry : entries) {
+ EntryAndMetadata entry;
+ if (inputEntry instanceof EntryAndMetadata entryAndMetadataInstance) {
+ entry = entryAndMetadataInstance;
+ } else {
+ // replace the input entry with EntryAndMetadata instance. In addition to the entry and metadata,
+ // it will also carry the calculated sticky key hash
+ entry = EntryAndMetadata.create(inputEntry,
+ Commands.peekAndCopyMessageMetadata(inputEntry.getDataBuffer(), getSubscriptionName(), -1));
+ }
int stickyKeyHash = getStickyKeyHash(entry);
Consumer consumer = selector.select(stickyKeyHash);
MutableBoolean blockedByHash = null;
@@ -460,15 +422,13 @@ private Map> filterAndGroupEntriesForDispatching(List resolveMaxLastSentPositionForRecentlyJoinedConsumer(consumer, readType)) : null;
blockedByHash = lookAheadAllowed && readType == ReadType.Normal ? new MutableBoolean(false) : null;
MutableInt permits =
permitsForConsumer.computeIfAbsent(consumer,
k -> new MutableInt(getAvailablePermits(consumer)));
// a consumer was found for the sticky key hash and the entry can be dispatched
- if (permits.intValue() > 0 && canDispatchEntry(entry, readType, stickyKeyHash,
- maxLastSentPosition, blockedByHash)) {
+ if (permits.intValue() > 0
+ && canDispatchEntry(consumer, entry, readType, stickyKeyHash, blockedByHash)) {
// decrement the permits for the consumer
permits.decrement();
// allow the entry to be dispatched
@@ -491,6 +451,7 @@ private Map> filterAndGroupEntriesForDispatching(List> filterAndGroupEntriesForDispatching(List 0) {
- return false;
- }
-
// If redeliveryMessages contains messages that correspond to the same hash as the entry to be dispatched
// do not send those messages for order guarantee
if (readType == ReadType.Normal && redeliveryMessages.containsStickyKeyHash(stickyKeyHash)) {
@@ -545,6 +501,16 @@ private boolean canDispatchEntry(Entry entry,
return false;
}
+ if (drainingHashesRequired) {
+ // If the hash is draining, do not send the message
+ if (drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash)) {
+ if (blockedByHash != null) {
+ blockedByHash.setTrue();
+ }
+ return false;
+ }
+ }
+
return true;
}
@@ -566,8 +532,6 @@ private class ReplayPositionFilter implements Predicate {
// tracks the available permits for each consumer for the duration of the filter usage
// the filter is stateful and shouldn't be shared or reused later
private final Map availablePermitsMap = new HashMap<>();
- private final Map maxLastSentPositionCache =
- hasRecentlyJoinedConsumers() ? new HashMap<>() : null;
@Override
public boolean test(Position position) {
@@ -585,6 +549,7 @@ public boolean test(Position position) {
}
return true;
}
+
// find the consumer for the sticky key hash
Consumer consumer = selector.select(stickyKeyHash.intValue());
// skip replaying the message position if there's no assigned consumer
@@ -599,113 +564,32 @@ public boolean test(Position position) {
if (availablePermits.intValue() <= 0) {
return false;
}
- // check if the entry position can be replayed to a recently joined consumer
- Position maxLastSentPosition = maxLastSentPositionCache != null
- ? maxLastSentPositionCache.computeIfAbsent(consumer, __ ->
- resolveMaxLastSentPositionForRecentlyJoinedConsumer(consumer, ReadType.Replay))
- : null;
- if (maxLastSentPosition != null && position.compareTo(maxLastSentPosition) > 0) {
+
+ if (drainingHashesRequired
+ && drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash.intValue())) {
+ // the hash is draining and the consumer is not the draining consumer
return false;
}
+
availablePermits.decrement();
return true;
}
}
- /**
- * Contains the logic to resolve the max last sent position for a consumer
- * when the consumer has recently joined. This is only applicable for key shared mode when
- * allowOutOfOrderDelivery=false.
- */
- private Position resolveMaxLastSentPositionForRecentlyJoinedConsumer(Consumer consumer, ReadType readType) {
- if (recentlyJoinedConsumers == null) {
- return null;
- }
- removeConsumersFromRecentJoinedConsumers();
- Position maxLastSentPosition = recentlyJoinedConsumers.get(consumer);
- // At this point, all the old messages were already consumed and this consumer
- // is now ready to receive any message
- if (maxLastSentPosition == null) {
- // The consumer has not recently joined, so we can send all messages
- return null;
- }
-
- // 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 minLastSentPositionForRecentJoinedConsumer = recentlyJoinedConsumers.values().iterator().next();
- if (minLastSentPositionForRecentJoinedConsumer != null
- && minLastSentPositionForRecentJoinedConsumer.compareTo(maxLastSentPosition) < 0) {
- maxLastSentPosition = minLastSentPositionForRecentJoinedConsumer;
- }
+ @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 maxLastSentPosition;
+ return selector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer()));
}
-
@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 (PersistentStickyKeyDispatcherMultipleConsumers.this) {
- if (hasRecentlyJoinedConsumers()
- && 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 hasRecentlyJoinedConsumers() {
- return !MapUtils.isEmpty(recentlyJoinedConsumers);
- }
-
- private boolean removeConsumersFromRecentJoinedConsumers() {
- if (MapUtils.isEmpty(recentlyJoinedConsumers)) {
- return false;
- }
- Iterator> itr = recentlyJoinedConsumers.entrySet().iterator();
- boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false;
- Position mdp = cursor.getMarkDeletedPosition();
- if (mdp != null) {
- while (itr.hasNext()) {
- Map.Entry entry = itr.next();
- if (entry.getValue().compareTo(mdp) <= 0) {
- itr.remove();
- hasConsumerRemovedFromTheRecentJoinedConsumers = true;
- } else {
- break;
- }
- }
- }
- return hasConsumerRemovedFromTheRecentJoinedConsumers;
- }
-
- @Nullable
- private synchronized Position updateIfNeededAndGetLastSentPosition() {
- if (lastSentPosition == null) {
- return null;
- }
- final Position mdp = cursor.getMarkDeletedPosition();
- if (mdp != null && mdp.compareTo(lastSentPosition) > 0) {
- lastSentPosition = mdp;
- }
- return lastSentPosition;
+ // reschedule a read with a backoff after moving the mark-delete position forward since there might have
+ // been consumers that were blocked by hash and couldn't make progress
+ reScheduleReadWithKeySharedUnblockingInterval();
}
/**
@@ -829,34 +713,6 @@ public boolean hasSameKeySharedPolicy(KeySharedMeta ksm) {
&& ksm.isAllowOutOfOrderDelivery() == this.allowOutOfOrderDelivery);
}
- public LinkedHashMap getRecentlyJoinedConsumers() {
- return recentlyJoinedConsumers;
- }
-
- public synchronized String getLastSentPosition() {
- if (lastSentPosition == null) {
- return null;
- }
- return lastSentPosition.toString();
- }
-
- @VisibleForTesting
- public Position getLastSentPositionField() {
- return lastSentPosition;
- }
-
- public synchronized String getIndividuallySentPositions() {
- if (individuallySentPositions == null) {
- return null;
- }
- return individuallySentPositions.toString();
- }
-
- @VisibleForTesting
- public LongPairRangeSet getIndividuallySentPositionsField() {
- return individuallySentPositions;
- }
-
public Map> getConsumerKeyHashRanges() {
return selector.getConsumerKeyHashRanges();
}
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 9a0545e6f0ab2..b8d351bddf839 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,7 +25,6 @@
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;
@@ -1297,31 +1296,6 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge
subStats.allowOutOfOrderDelivery = keySharedDispatcher.isAllowOutOfOrderDelivery();
subStats.keySharedMode = keySharedDispatcher.getKeySharedMode().toString();
-
- LinkedHashMap recentlyJoinedConsumers = keySharedDispatcher
- .getRecentlyJoinedConsumers();
- if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) {
- recentlyJoinedConsumers.forEach((k, v) -> {
- // The dispatcher allows same name consumers
- final StringBuilder stringBuilder = new StringBuilder();
- stringBuilder.append("consumerName=").append(k.consumerName())
- .append(", consumerId=").append(k.consumerId());
- if (k.cnx() != null) {
- stringBuilder.append(", address=").append(k.cnx().clientAddress());
- }
- subStats.consumersAfterMarkDeletePosition.put(stringBuilder.toString(), v.toString());
- });
- }
- final String lastSentPosition = ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher)
- .getLastSentPosition();
- if (lastSentPosition != null) {
- subStats.lastSentPosition = lastSentPosition;
- }
- final String individuallySentPositions = ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher)
- .getIndividuallySentPositions();
- if (individuallySentPositions != null) {
- subStats.individuallySentPositions = individuallySentPositions;
- }
}
subStats.nonContiguousDeletedMessagesRanges = cursor.getTotalNonContiguousDeletedMessagesRange();
subStats.nonContiguousDeletedMessagesRangesSerializedSize =
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java
new file mode 100644
index 0000000000000..3554f29255227
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java
@@ -0,0 +1,102 @@
+/*
+ * 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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BooleanSupplier;
+import java.util.function.LongSupplier;
+
+/**
+ * Reschedules reads so that the possible pending read is cancelled if it's waiting for more entries.
+ * This will prevent the dispatcher in getting blocked when there are entries in the replay queue
+ * that should be handled. This will also batch multiple calls together to reduce the number of
+ * operations.
+ */
+class RescheduleReadHandler {
+ private static final int UNSET = -1;
+ private static final int NO_PENDING_READ = 0;
+ private final AtomicLong maxReadOpCounter = new AtomicLong(UNSET);
+ private final LongSupplier readIntervalMsSupplier;
+ private final ScheduledExecutorService executor;
+ private final Runnable cancelPendingRead;
+ private final Runnable rescheduleReadImmediately;
+ private final BooleanSupplier hasPendingReadRequestThatMightWait;
+ private final LongSupplier readOpCounterSupplier;
+ private final BooleanSupplier hasEntriesInReplayQueue;
+
+ RescheduleReadHandler(LongSupplier readIntervalMsSupplier,
+ ScheduledExecutorService executor, Runnable cancelPendingRead,
+ Runnable rescheduleReadImmediately, BooleanSupplier hasPendingReadRequestThatMightWait,
+ LongSupplier readOpCounterSupplier,
+ BooleanSupplier hasEntriesInReplayQueue) {
+ this.readIntervalMsSupplier = readIntervalMsSupplier;
+ this.executor = executor;
+ this.cancelPendingRead = cancelPendingRead;
+ this.rescheduleReadImmediately = rescheduleReadImmediately;
+ this.hasPendingReadRequestThatMightWait = hasPendingReadRequestThatMightWait;
+ this.readOpCounterSupplier = readOpCounterSupplier;
+ this.hasEntriesInReplayQueue = hasEntriesInReplayQueue;
+ }
+
+ public void rescheduleRead() {
+ long readOpCountWhenPendingRead =
+ hasPendingReadRequestThatMightWait.getAsBoolean() ? readOpCounterSupplier.getAsLong() : NO_PENDING_READ;
+ if (maxReadOpCounter.compareAndSet(UNSET, readOpCountWhenPendingRead)) {
+ Runnable runnable = () -> {
+ // Read the current value of maxReadOpCounter and set it to UNSET, this will allow scheduling a next
+ // runnable
+ long maxReadOpCount = maxReadOpCounter.getAndSet(UNSET);
+ // Cancel a possible pending read if it's been waiting for more entries since the runnable was
+ // scheduled. This is detected by checking that the value of the readOpCounter has not changed
+ // since the runnable was scheduled. Canceling the read request will only be needed if there
+ // are entries in the replay queue.
+ if (maxReadOpCount != NO_PENDING_READ && readOpCounterSupplier.getAsLong() == maxReadOpCount
+ && hasEntriesInReplayQueue.getAsBoolean()) {
+ cancelPendingRead.run();
+ }
+ // Re-schedule read immediately, or join the next scheduled read
+ rescheduleReadImmediately.run();
+ };
+ long rescheduleDelay = readIntervalMsSupplier.getAsLong();
+ if (rescheduleDelay > 0) {
+ executor.schedule(runnable, rescheduleDelay, TimeUnit.MILLISECONDS);
+ } else {
+ runnable.run();
+ }
+ } else {
+ // When there's a scheduled read, update the maxReadOpCounter to carry the state when the later scheduled
+ // read was done
+ long updatedValue = maxReadOpCounter.updateAndGet(
+ // Ignore updating if the value is UNSET
+ current -> current == UNSET ? UNSET :
+ // Prefer keeping NO_PENDING_READ if the latest value is NO_PENDING_READ
+ (readOpCountWhenPendingRead == NO_PENDING_READ ? NO_PENDING_READ :
+ // Otherwise, keep the maximum value
+ Math.max(current, readOpCountWhenPendingRead)));
+ // If the value was unset, it means that the runnable was already run and retrying is needed
+ // so that we don't miss any entries
+ if (updatedValue == UNSET) {
+ // Retry
+ rescheduleRead();
+ }
+ }
+ }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java
index cc1eae475fa2d..7a4126fedec64 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java
@@ -71,19 +71,30 @@ public boolean contains(long item1, long item2) {
}
}
- public void removeUpTo(long item1, long item2) {
+ /**
+ * Remove all items up to (and including) the specified item.
+ *
+ * @param item1 the first part of the item key
+ * @param item2 the second part of the item key
+ * @return true if any bits were cleared
+ */
+ public boolean removeUpTo(long item1, long item2) {
+ boolean bitsCleared = false;
lock.writeLock().lock();
try {
Map.Entry firstEntry = map.firstEntry();
while (firstEntry != null && firstEntry.getKey() <= item1) {
if (firstEntry.getKey() < item1) {
map.remove(firstEntry.getKey(), firstEntry.getValue());
+ bitsCleared = true;
} else {
RoaringBitmap bitSet = firstEntry.getValue();
if (bitSet != null) {
+ bitsCleared |= bitSet.contains(0, item2);
bitSet.remove(0, item2);
if (bitSet.isEmpty()) {
map.remove(firstEntry.getKey(), bitSet);
+ bitsCleared = true;
}
}
break;
@@ -93,6 +104,7 @@ public void removeUpTo(long item1, long item2) {
} finally {
lock.writeLock().unlock();
}
+ return bitsCleared;
}
public > Optional first(LongPairSet.LongPairFunction longPairConverter) {
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 5641816ee0b80..ffcc3bf0881db 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
@@ -18,6 +18,8 @@
*/
package org.apache.pulsar.broker;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.ObjectWriter;
import java.io.IOException;
@@ -234,4 +236,21 @@ public static void receiveMessagesInThreads(BiFunction, Message<
});
}).toList()).join();
}
+
+ private static long mockConsumerIdGenerator = 0;
+
+ public static org.apache.pulsar.broker.service.Consumer createMockConsumer(String consumerName) {
+ long consumerId = mockConsumerIdGenerator++;
+ return createMockConsumer(consumerName, consumerName + " consumerId:" + consumerId, consumerId);
+ }
+
+ public static org.apache.pulsar.broker.service.Consumer createMockConsumer(String consumerName, String toString, long consumerId) {
+ // without stubOnly, the mock will record method invocations and could run into OOME
+ org.apache.pulsar.broker.service.Consumer
+ consumer = mock(org.apache.pulsar.broker.service.Consumer.class, Mockito.withSettings().stubOnly());
+ when(consumer.consumerName()).thenReturn(consumerName);
+ when(consumer.toString()).thenReturn(consumerName + " consumerId:" + consumerId);
+ when(consumer.consumerId()).thenReturn(consumerId);
+ return consumer;
+ }
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
index 4a1dbface2c63..ed015592b8c76 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
@@ -18,8 +18,6 @@
*/
package org.apache.pulsar.broker.admin;
-import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
@@ -58,7 +56,6 @@
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Function;
import javax.ws.rs.client.InvocationCallback;
import javax.ws.rs.client.WebTarget;
import javax.ws.rs.core.Response.Status;
@@ -68,8 +65,6 @@
import lombok.Value;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.ManagedLedgerInfo;
-import org.apache.bookkeeper.mledger.Position;
-import org.apache.bookkeeper.mledger.PositionFactory;
import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.PulsarService;
@@ -79,8 +74,6 @@
import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl;
import org.apache.pulsar.broker.namespace.NamespaceEphemeralData;
import org.apache.pulsar.broker.namespace.NamespaceService;
-import org.apache.pulsar.broker.service.StickyKeyConsumerSelector;
-import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers;
import org.apache.pulsar.broker.testcontext.PulsarTestContext;
import org.apache.pulsar.broker.testcontext.SpyConfig;
import org.apache.pulsar.client.admin.GetStatsOptions;
@@ -128,7 +121,6 @@
import org.apache.pulsar.common.policies.data.BrokerInfo;
import org.apache.pulsar.common.policies.data.BundlesData;
import org.apache.pulsar.common.policies.data.ClusterData;
-import org.apache.pulsar.common.policies.data.ConsumerStats;
import org.apache.pulsar.common.policies.data.NamespaceIsolationData;
import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl;
import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus;
@@ -145,10 +137,7 @@
import org.apache.pulsar.common.policies.data.TopicHashPositions;
import org.apache.pulsar.common.policies.data.TopicStats;
import org.apache.pulsar.common.util.Codec;
-import org.apache.pulsar.common.util.Murmur3_32Hash;
import org.apache.pulsar.common.util.ObjectMapperFactory;
-import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet;
-import org.apache.pulsar.common.util.collections.LongPairRangeSet;
import org.apache.pulsar.compaction.Compactor;
import org.apache.pulsar.compaction.PulsarCompactionServiceFactory;
import org.awaitility.Awaitility;
@@ -3457,201 +3446,6 @@ public void testGetTtlDurationDefaultInSeconds() throws Exception {
assertNull(seconds);
}
- @Test
- public void testGetLastSentPositionWhenJoining() throws Exception {
- final String topic = "persistent://prop-xyz/ns1/testGetLastSentPositionWhenJoining-" + UUID.randomUUID().toString();
- final String subName = "my-sub";
- @Cleanup
- Producer producer = pulsarClient.newProducer()
- .topic(topic)
- .enableBatching(false)
- .create();
-
- @Cleanup
- final Consumer consumer1 = pulsarClient.newConsumer()
- .topic(topic)
- .subscriptionType(SubscriptionType.Key_Shared)
- .subscriptionName(subName)
- .subscribe();
-
- final int messages = 10;
- MessageIdImpl messageId = null;
- for (int i = 0; i < messages; i++) {
- messageId = (MessageIdImpl) producer.send(("Hello Pulsar - " + i).getBytes());
- consumer1.receive();
- }
-
- @Cleanup
- final Consumer consumer2 = pulsarClient.newConsumer()
- .topic(topic)
- .subscriptionType(SubscriptionType.Key_Shared)
- .subscriptionName(subName)
- .subscribe();
-
- TopicStats stats = admin.topics().getStats(topic);
- Assert.assertEquals(stats.getSubscriptions().size(), 1);
- SubscriptionStats subStats = stats.getSubscriptions().get(subName);
- Assert.assertNotNull(subStats);
- Assert.assertEquals(subStats.getConsumers().size(), 2);
- ConsumerStats consumerStats = subStats.getConsumers().stream()
- .filter(s -> s.getConsumerName().equals(consumer2.getConsumerName())).findFirst().get();
- Assert.assertEquals(consumerStats.getLastSentPositionWhenJoining(),
- PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()).toString());
- }
-
- @Test
- public void testGetLastSentPosition() throws Exception {
- final String topic = "persistent://prop-xyz/ns1/testGetLastSentPosition-" + UUID.randomUUID().toString();
- final String subName = "my-sub";
- @Cleanup
- final Producer producer = pulsarClient.newProducer()
- .topic(topic)
- .enableBatching(false)
- .create();
- final AtomicInteger counter = new AtomicInteger();
- @Cleanup
- final Consumer consumer = pulsarClient.newConsumer()
- .topic(topic)
- .subscriptionType(SubscriptionType.Key_Shared)
- .subscriptionName(subName)
- .messageListener((c, msg) -> {
- try {
- c.acknowledge(msg);
- counter.getAndIncrement();
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- })
- .subscribe();
-
- TopicStats stats = admin.topics().getStats(topic);
- Assert.assertEquals(stats.getSubscriptions().size(), 1);
- SubscriptionStats subStats = stats.getSubscriptions().get(subName);
- Assert.assertNotNull(subStats);
- Assert.assertNull(subStats.getLastSentPosition());
-
- final int messages = 10;
- MessageIdImpl messageId = null;
- for (int i = 0; i < messages; i++) {
- messageId = (MessageIdImpl) producer.send(("Hello Pulsar - " + i).getBytes());
- }
-
- Awaitility.await().untilAsserted(() -> assertEquals(counter.get(), messages));
-
- stats = admin.topics().getStats(topic);
- Assert.assertEquals(stats.getSubscriptions().size(), 1);
- subStats = stats.getSubscriptions().get(subName);
- Assert.assertNotNull(subStats);
- Assert.assertEquals(subStats.getLastSentPosition(), PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()).toString());
- }
-
- @Test
- public void testGetIndividuallySentPositions() throws Exception {
- // The producer sends messages with two types of keys.
- // The dispatcher sends keyA messages to consumer1.
- // Consumer1 will not receive any messages. Its receiver queue size is 1.
- // Consumer2 will receive and ack any messages immediately.
-
- final String topic = "persistent://prop-xyz/ns1/testGetIndividuallySentPositions-" + UUID.randomUUID().toString();
- final String subName = "my-sub";
- @Cleanup
- final Producer producer = pulsarClient.newProducer()
- .topic(topic)
- .enableBatching(false)
- .create();
-
- final String consumer1Name = "c1";
- final String consumer2Name = "c2";
-
- @Cleanup
- final Consumer consumer1 = pulsarClient.newConsumer()
- .topic(topic)
- .consumerName(consumer1Name)
- .receiverQueueSize(1)
- .subscriptionType(SubscriptionType.Key_Shared)
- .subscriptionName(subName)
- .subscribe();
-
- final PersistentStickyKeyDispatcherMultipleConsumers dispatcher =
- (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get().get().getSubscription(subName).getDispatcher();
- final String keyA = "key-a";
- final String keyB = "key-b";
- final int hashA = Murmur3_32Hash.getInstance().makeHash(keyA.getBytes());
-
- final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector");
- selectorField.setAccessible(true);
- final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher));
- selectorField.set(dispatcher, selector);
-
- // the selector returns consumer1 if keyA
- doAnswer((invocationOnMock -> {
- final int hash = invocationOnMock.getArgument(0);
-
- final String consumerName = hash == hashA ? consumer1Name : consumer2Name;
- return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get();
- })).when(selector).select(anyInt());
-
- final AtomicInteger consumer2AckCounter = new AtomicInteger();
- @Cleanup
- final Consumer consumer2 = pulsarClient.newConsumer()
- .topic(topic)
- .consumerName(consumer2Name)
- .subscriptionType(SubscriptionType.Key_Shared)
- .subscriptionName(subName)
- .messageListener((c, msg) -> {
- try {
- c.acknowledge(msg);
- consumer2AckCounter.getAndIncrement();
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- })
- .subscribe();
-
- final LongPairRangeSet.LongPairConsumer positionRangeConverter = PositionFactory::create;
- final LongPairRangeSet expectedIndividuallySentPositions = new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter);
-
- TopicStats stats = admin.topics().getStats(topic);
- Assert.assertEquals(stats.getSubscriptions().size(), 1);
- SubscriptionStats subStats = stats.getSubscriptions().get(subName);
- Assert.assertNotNull(subStats);
- Assert.assertEquals(subStats.getIndividuallySentPositions(), expectedIndividuallySentPositions.toString());
-
- final Function sendFn = (key) -> {
- try {
- return (MessageIdImpl) producer.newMessage().key(key).value(("msg").getBytes()).send();
- } catch (PulsarClientException e) {
- throw new RuntimeException(e);
- }
- };
- final List messageIdList = new ArrayList<>();
-
- // the dispatcher can send keyA message, but then consumer1's receiver queue will be full
- messageIdList.add(sendFn.apply(keyA));
-
- // the dispatcher can send messages other than keyA
- messageIdList.add(sendFn.apply(keyA));
- messageIdList.add(sendFn.apply(keyB));
- messageIdList.add(sendFn.apply(keyA));
- messageIdList.add(sendFn.apply(keyB));
- messageIdList.add(sendFn.apply(keyB));
-
- assertEquals(messageIdList.size(), 6);
- Awaitility.await().untilAsserted(() -> assertEquals(consumer2AckCounter.get(), 3));
-
- // set expected value
- expectedIndividuallySentPositions.addOpenClosed(messageIdList.get(1).getLedgerId(), messageIdList.get(1).getEntryId(),
- messageIdList.get(2).getLedgerId(), messageIdList.get(2).getEntryId());
- expectedIndividuallySentPositions.addOpenClosed(messageIdList.get(3).getLedgerId(), messageIdList.get(3).getEntryId(),
- messageIdList.get(5).getLedgerId(), messageIdList.get(5).getEntryId());
-
- stats = admin.topics().getStats(topic);
- Assert.assertEquals(stats.getSubscriptions().size(), 1);
- subStats = stats.getSubscriptions().get(subName);
- Assert.assertNotNull(subStats);
- Assert.assertEquals(subStats.getIndividuallySentPositions(), expectedIndividuallySentPositions.toString());
- }
-
@Test
public void testPartitionedTopicMsgDelayedAggregated() throws Exception {
final String topic = "persistent://prop-xyz/ns1/testPartitionedTopicMsgDelayedAggregated-" + UUID.randomUUID().toString();
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 8b72411329c65..ea6ffa2d70dba 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
@@ -126,10 +126,10 @@ public void testWithTimer(DelayedDeliveryTracker tracker, NavigableMap false);
((AbstractDelayedDeliveryTracker) tracker).run(timeout);
- verify(dispatcher, times(1)).readMoreEntries();
+ verify(dispatcher, times(1)).readMoreEntriesAsync();
// Add a message that has a delivery time just after the previous run. It will get delivered based on the
// tick delay plus the last tick run.
@@ -189,11 +189,11 @@ public void testAddMessageWithDeliverAtTimeAfterNowBeforeTickTimeFrequencyWithSt
// Wait longer than the tick time plus the HashedWheelTimer's tick time to ensure that enough time has
// passed where it would have been triggered if the tick time was doing the triggering.
Thread.sleep(600);
- verify(dispatcher, times(1)).readMoreEntries();
+ verify(dispatcher, times(1)).readMoreEntriesAsync();
// Not wait for the message delivery to get triggered.
Awaitility.await().atMost(10, TimeUnit.SECONDS)
- .untilAsserted(() -> verify(dispatcher).readMoreEntries());
+ .untilAsserted(() -> verify(dispatcher).readMoreEntriesAsync());
tracker.close();
}
@@ -212,7 +212,7 @@ public void testAddMessageWithDeliverAtTimeAfterNowAfterTickTimeFrequencyWithStr
// Wait long enough for the runnable to run, but not longer than the tick time. The point is that the delivery
// should get scheduled early when the tick duration has passed since the last tick.
Awaitility.await().atMost(10, TimeUnit.SECONDS)
- .untilAsserted(() -> verify(dispatcher).readMoreEntries());
+ .untilAsserted(() -> verify(dispatcher).readMoreEntriesAsync());
tracker.close();
}
@@ -233,7 +233,7 @@ public void testAddMessageWithDeliverAtTimeAfterFullTickTimeWithStrict(DelayedDe
// Not wait for the message delivery to get triggered.
Awaitility.await().atMost(10, TimeUnit.SECONDS)
- .untilAsserted(() -> verify(dispatcher).readMoreEntries());
+ .untilAsserted(() -> verify(dispatcher).readMoreEntriesAsync());
tracker.close();
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java
index 04aafc49b47e6..e2feb2050652b 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java
@@ -168,20 +168,20 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume
Map> expectedResult = new HashMap<>();
assertThat(consumers.get(0).consumerName()).isEqualTo("consumer1");
expectedResult.put(consumers.get(0), Arrays.asList(
- Range.of(95615213, 440020355),
- Range.of(440020356, 455987436),
- Range.of(1189794593, 1264144431)));
+ Range.of(14359, 18366),
+ Range.of(29991, 39817),
+ Range.of(52980, 60442)));
assertThat(consumers.get(1).consumerName()).isEqualTo("consumer2");
expectedResult.put(consumers.get(1), Arrays.asList(
- Range.of(939655188, 1189794592),
- Range.of(1314727625, 1977451233),
- Range.of(1977451234, 2016237253)));
+ Range.of(1, 6668),
+ Range.of(39818, 52979),
+ Range.of(60443, 63679),
+ Range.of(65184, 65535)));
assertThat(consumers.get(2).consumerName()).isEqualTo("consumer3");
expectedResult.put(consumers.get(2), Arrays.asList(
- Range.of(0, 95615212),
- Range.of(455987437, 939655187),
- Range.of(1264144432, 1314727624),
- Range.of(2016237254, 2147483646)));
+ Range.of(6669, 14358),
+ Range.of(18367, 29990),
+ Range.of(63680, 65183)));
Map> consumerKeyHashRanges = selector.getConsumerKeyHashRanges();
assertThat(consumerKeyHashRanges).containsExactlyInAnyOrderEntriesOf(expectedResult);
@@ -195,7 +195,8 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume
}
previousRange = range;
}
- assertThat(allRanges.stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum()).isEqualTo(Integer.MAX_VALUE);
+ Range totalRange = selector.getKeyHashRange();
+ assertThat(allRanges.stream().mapToInt(Range::size).sum()).isEqualTo(totalRange.size());
}
@Test
@@ -247,12 +248,12 @@ private static void printSelectionCountStats(Map consumerS
private static void printConsumerRangesStats(ConsistentHashingStickyKeyConsumerSelector selector) {
selector.getConsumerKeyHashRanges().entrySet().stream()
.map(entry -> Map.entry(entry.getKey(),
- entry.getValue().stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum()))
+ entry.getValue().stream().mapToInt(Range::size).sum()))
.sorted(Map.Entry.comparingByKey(Comparator.comparing(Consumer::toString)))
.forEach(entry -> System.out.println(
String.format("consumer: %s total ranges size: %d ratio: %.2f%%", entry.getKey(),
entry.getValue(),
- ((double) entry.getValue() / (Integer.MAX_VALUE - 1)) * 100.0d)));
+ ((double) entry.getValue() / selector.getKeyHashRange().size()) * 100.0d)));
}
private static Consumer createMockConsumer(String consumerName, String toString, long id) {
@@ -323,7 +324,7 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemoved() {
selector.addConsumer(consumer);
}
- int hashRangeSize = Integer.MAX_VALUE;
+ int hashRangeSize = selector.getKeyHashRange().size();
int validationPointCount = 200;
int increment = hashRangeSize / (validationPointCount + 1);
List selectedConsumerBeforeRemoval = new ArrayList<>();
@@ -342,13 +343,14 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemoved() {
for (Consumer removedConsumer : consumers) {
selector.removeConsumer(removedConsumer);
removedConsumers.add(removedConsumer);
+ Map> consumerKeyHashRanges = selector.getConsumerKeyHashRanges();
for (int i = 0; i < validationPointCount; i++) {
int hash = i * increment;
Consumer selected = selector.select(hash);
Consumer expected = selectedConsumerBeforeRemoval.get(i);
if (!removedConsumers.contains(expected)) {
assertThat(selected.consumerId()).as("validationPoint %d, removed %s, hash %d ranges %s", i,
- removedConsumer.toString(), hash, selector.getConsumerKeyHashRanges()).isEqualTo(expected.consumerId());
+ removedConsumer.toString(), hash, consumerKeyHashRanges).isEqualTo(expected.consumerId());
}
}
}
@@ -441,7 +443,7 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() {
selector.addConsumer(consumer);
}
- int hashRangeSize = Integer.MAX_VALUE;
+ int hashRangeSize = selector.getKeyHashRange().size();
int validationPointCount = 200;
int increment = hashRangeSize / (validationPointCount + 1);
List selectedConsumerBeforeRemoval = new ArrayList<>();
@@ -473,10 +475,10 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() {
}
@Test
- public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() {
+ public void testShouldContainMinimalMappingChangesWhenConsumerLeavesAndRejoins() {
final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100);
final String consumerName = "consumer";
- final int numOfInitialConsumers = 25;
+ final int numOfInitialConsumers = 10;
List consumers = new ArrayList<>();
for (int i = 0; i < numOfInitialConsumers; i++) {
final Consumer consumer = createMockConsumer(consumerName, "index " + i, i);
@@ -484,6 +486,8 @@ public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() {
selector.addConsumer(consumer);
}
+ ConsumerHashAssignmentsSnapshot assignmentsBefore = selector.getConsumerHashAssignmentsSnapshot();
+
Map> expected = selector.getConsumerKeyHashRanges();
assertThat(selector.getConsumerKeyHashRanges()).as("sanity check").containsExactlyInAnyOrderEntriesOf(expected);
@@ -492,7 +496,15 @@ public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() {
selector.addConsumer(consumers.get(0));
selector.addConsumer(consumers.get(numOfInitialConsumers / 2));
- assertThat(selector.getConsumerKeyHashRanges()).as("ranges shouldn't change").containsExactlyInAnyOrderEntriesOf(expected);
+ ConsumerHashAssignmentsSnapshot assignmentsAfter = selector.getConsumerHashAssignmentsSnapshot();
+ int removedRangesSize = assignmentsBefore.diffRanges(assignmentsAfter).keySet().stream()
+ .mapToInt(Range::size)
+ .sum();
+ double allowedremovedRangesPercentage = 1; // 1%
+ int hashRangeSize = selector.getKeyHashRange().size();
+ int allowedremovedRanges = (int) (hashRangeSize * (allowedremovedRangesPercentage / 100.0d));
+ assertThat(removedRangesSize).describedAs("Allow up to %d%% of total hash range size to be impacted",
+ allowedremovedRangesPercentage).isLessThan(allowedremovedRanges);
}
@Test
@@ -501,7 +513,7 @@ public void testConsumersReconnect() {
final String consumerName = "consumer";
final int numOfInitialConsumers = 50;
final int validationPointCount = 200;
- final List pointsToTest = pointsToTest(validationPointCount);
+ final List pointsToTest = pointsToTest(validationPointCount, selector.getKeyHashRange().size());
List consumers = new ArrayList<>();
for (int i = 0; i < numOfInitialConsumers; i++) {
final Consumer consumer = createMockConsumer(consumerName, "index " + i, i);
@@ -537,13 +549,38 @@ public void testConsumersReconnect() {
}
}
- private List pointsToTest(int validationPointCount) {
+ private List pointsToTest(int validationPointCount, int hashRangeSize) {
List res = new ArrayList<>();
- int hashRangeSize = Integer.MAX_VALUE;
final int increment = hashRangeSize / (validationPointCount + 1);
for (int i = 0; i < validationPointCount; i++) {
- res.add(i * increment);
+ res.add(Math.max(i * increment, hashRangeSize - 1));
}
return res;
}
-}
+
+ @Test(enabled = false)
+ public void testPerformanceOfAdding1000ConsumersWith100Points() {
+ // test that adding 1000 consumers with 100 points runs in a reasonable time.
+ // This takes about 1 second on Apple M3
+ // this unit test can be used for basic profiling
+ final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100);
+ for (int i = 0; i < 1000; i++) {
+ // use real class to avoid Mockito over head
+ final Consumer consumer = new Consumer("consumer" + i, 0) {
+ @Override
+ public int hashCode() {
+ return consumerName().hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof Consumer) {
+ return consumerName().equals(((Consumer) obj).consumerName());
+ }
+ return false;
+ }
+ };
+ selector.addConsumer(consumer);
+ }
+ }
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java
new file mode 100644
index 0000000000000..5c886b6eec9f3
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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 static org.apache.pulsar.broker.BrokerTestUtil.createMockConsumer;
+import static org.assertj.core.api.Assertions.assertThat;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.client.api.Range;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker-impl")
+public class ConsumerHashAssignmentsSnapshotTest {
+ @Test
+ public void testMergeOverlappingRanges() {
+ SortedSet ranges = new TreeSet<>();
+ ranges.add(Range.of(1, 5));
+ ranges.add(Range.of(6, 10));
+ ranges.add(Range.of(8, 12));
+ ranges.add(Range.of(15, 20));
+ ranges.add(Range.of(21, 25));
+
+ SortedSet expectedMergedRanges = new TreeSet<>();
+ expectedMergedRanges.add(Range.of(1, 12));
+ expectedMergedRanges.add(Range.of(15, 25));
+
+ List mergedRanges = ConsumerHashAssignmentsSnapshot.mergeOverlappingRanges(ranges);
+
+ assertThat(mergedRanges).containsExactlyElementsOf(expectedMergedRanges);
+ }
+
+ @Test
+ public void testDiffRanges_NoChanges() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+
+ Consumer consumer1 = createMockConsumer("consumer1");
+ mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+ mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+
+ Map> diff =
+ ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter);
+
+ assertThat(diff).isEmpty();
+ }
+
+ @Test
+ public void testDiffRanges_ConsumerChanged() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+
+ Consumer consumer1 = createMockConsumer("consumer1");
+ Consumer consumer2 = createMockConsumer("consumer2");
+ mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+ mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer2));
+
+ Map> diff =
+ ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter);
+
+ assertThat(diff).containsEntry(Range.of(1, 5), Pair.of(consumer1, consumer2));
+ }
+
+ @Test
+ public void testDiffRanges_RangeAdded() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+ Consumer consumer1 = createMockConsumer("consumer1");
+
+ mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+
+ Map> diff =
+ ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter);
+
+ assertThat(diff).containsEntry(Range.of(1, 5), Pair.of(null, consumer1));
+ }
+
+ @Test
+ public void testDiffRanges_RangeRemoved() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+
+ Consumer consumer1 = createMockConsumer("consumer1");
+ mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+
+ Map> diff =
+ ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter);
+
+ assertThat(diff).containsEntry(Range.of(1, 5), Pair.of(consumer1, null));
+ }
+
+ @Test
+ public void testDiffRanges_OverlappingRanges() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+
+ Consumer consumer1 = createMockConsumer("consumer1");
+ Consumer consumer2 = createMockConsumer("consumer2");
+ mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+ mappingAfter.add(new HashRangeAssignment(Range.of(3, 7), consumer2));
+
+ Map> diff =
+ ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter);
+
+ assertThat(diff).containsEntry(Range.of(3, 5), Pair.of(consumer1, consumer2));
+ }
+
+ @Test
+ public void testResolveConsumerRemovedHashRanges_NoChanges() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+
+ Consumer consumer1 = createMockConsumer("consumer1");
+ mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+ mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+
+ ImpactedConsumersResult impactedConsumers =
+ ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter);
+
+ assertThat(impactedConsumers.getRemovedHashRanges()).isEmpty();
+ }
+
+ @Test
+ public void testResolveConsumerRemovedHashRanges_ConsumerChanged() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+
+ Consumer consumer1 = createMockConsumer("consumer1");
+ Consumer consumer2 = createMockConsumer("consumer2");
+ mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+ mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer2));
+
+ ImpactedConsumersResult impactedConsumers =
+ ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter);
+
+ assertThat(impactedConsumers.getRemovedHashRanges()).containsExactlyInAnyOrderEntriesOf(
+ Map.of(consumer1, RemovedHashRanges.of(List.of(Range.of(1, 5)))));
+ }
+
+ @Test
+ public void testResolveConsumerRemovedHashRanges_RangeAdded() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+
+ Consumer consumer1 = createMockConsumer("consumer1");
+ mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+
+ ImpactedConsumersResult impactedConsumers =
+ ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter);
+
+ assertThat(impactedConsumers.getRemovedHashRanges()).isEmpty();
+ }
+
+ @Test
+ public void testResolveConsumerRemovedHashRanges_RangeRemoved() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+
+ Consumer consumer1 = createMockConsumer("consumer1");
+ mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+
+ ImpactedConsumersResult impactedConsumers =
+ ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter);
+
+ assertThat(impactedConsumers.getRemovedHashRanges()).containsExactlyInAnyOrderEntriesOf(
+ Map.of(consumer1, RemovedHashRanges.of(List.of(Range.of(1, 5)))));
+ }
+
+ @Test
+ public void testResolveConsumerRemovedHashRanges_OverlappingRanges() {
+ List mappingBefore = new ArrayList<>();
+ List mappingAfter = new ArrayList<>();
+
+ Consumer consumer1 = createMockConsumer("consumer1");
+ Consumer consumer2 = createMockConsumer("consumer2");
+ mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1));
+ mappingAfter.add(new HashRangeAssignment(Range.of(3, 7), consumer2));
+
+ ImpactedConsumersResult impactedConsumers =
+ ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter);
+
+ assertThat(impactedConsumers.getRemovedHashRanges()).containsExactlyInAnyOrderEntriesOf(
+ Map.of(consumer1, RemovedHashRanges.of(List.of(Range.of(3, 5)))));
+ }
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java
new file mode 100644
index 0000000000000..ecb20beeb648a
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java
@@ -0,0 +1,213 @@
+/*
+ * 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 static org.apache.pulsar.broker.BrokerTestUtil.createMockConsumer;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertSame;
+import static org.testng.Assert.assertThrows;
+import static org.testng.Assert.assertTrue;
+import org.apache.pulsar.broker.service.DrainingHashesTracker.UnblockingHandler;
+import org.testng.annotations.Test;
+
+public class DrainingHashesTrackerTest {
+ @Test
+ public void addEntry_AddsNewEntry() {
+ Consumer consumer = createMockConsumer("consumer1");
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+
+ tracker.addEntry(consumer, 1);
+
+ assertNotNull(tracker.getEntry(1));
+ assertSame(tracker.getEntry(1).getConsumer(), consumer);
+ }
+
+ @Test
+ public void addEntry_ThrowsExceptionForZeroStickyHash() {
+ Consumer consumer = createMockConsumer("consumer1");
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+
+ assertThrows(IllegalArgumentException.class, () -> tracker.addEntry(consumer, 0));
+ }
+
+ @Test
+ public void reduceRefCount_ReducesReferenceCount() {
+ Consumer consumer = createMockConsumer("consumer1");
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+ tracker.addEntry(consumer, 1);
+
+ tracker.reduceRefCount(consumer, 1, false);
+
+ assertNull(tracker.getEntry(1));
+ }
+
+ @Test
+ public void reduceRefCount_DoesNotReduceForDifferentConsumer() {
+ Consumer consumer1 = createMockConsumer("consumer1");
+ Consumer consumer2 = createMockConsumer("consumer2");
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+ tracker.addEntry(consumer1, 1);
+
+ assertThrows(IllegalStateException.class, () -> tracker.reduceRefCount(consumer2, 1, false));
+
+ assertNotNull(tracker.getEntry(1));
+ assertSame(tracker.getEntry(1).getConsumer(), consumer1);
+ }
+
+ @Test
+ public void shouldBlockStickyKeyHash_DoesNotBlockForExistingEntryWhenSameConsumer() {
+ Consumer consumer = createMockConsumer("consumer1");
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+ tracker.addEntry(consumer, 1);
+
+ boolean result = tracker.shouldBlockStickyKeyHash(consumer, 1);
+
+ assertFalse(result);
+ }
+
+ @Test
+ public void shouldBlockStickyKeyHash_BlocksForExistingEntryWhenDifferentConsumer() {
+ Consumer consumer1 = createMockConsumer("consumer1");
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+ tracker.addEntry(consumer1, 1);
+
+ Consumer consumer2 = createMockConsumer("consumer2");
+ boolean result = tracker.shouldBlockStickyKeyHash(consumer2, 1);
+
+ assertTrue(result);
+ }
+
+
+ @Test
+ public void shouldBlockStickyKeyHash_DoesNotBlockForNewEntry() {
+ Consumer consumer = createMockConsumer("consumer1");
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+
+ boolean result = tracker.shouldBlockStickyKeyHash(consumer, 1);
+
+ assertFalse(result);
+ }
+
+ @Test
+ public void startBatch_IncrementsBatchLevel() {
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+
+ tracker.startBatch();
+ assertEquals(tracker.batchLevel, 1);
+
+ tracker.startBatch();
+ assertEquals(tracker.batchLevel, 2);
+
+ tracker.startBatch();
+ assertEquals(tracker.batchLevel, 3);
+ }
+
+ @Test
+ public void endBatch_DecrementsBatchLevel() {
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+ tracker.startBatch();
+
+ tracker.endBatch();
+
+ assertEquals(tracker.batchLevel, 0);
+ }
+
+ @Test
+ public void endBatch_InvokesUnblockingHandlerWhenUnblockedWhileBatching() {
+ // given a tracker with unblocking handler
+ UnblockingHandler unblockingHandler = mock(UnblockingHandler.class);
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", unblockingHandler);
+
+ // when a hash is draining
+ Consumer consumer1 = createMockConsumer("consumer1");
+ tracker.addEntry(consumer1, 1);
+ // and batch starts
+ tracker.startBatch();
+
+ // when hash gets blocked
+ Consumer consumer2 = createMockConsumer("consumer2");
+ tracker.shouldBlockStickyKeyHash(consumer2, 1);
+ // and it gets unblocked
+ tracker.reduceRefCount(consumer1, 1, false);
+
+ // then no unblocking call should be done
+ verify(unblockingHandler, never()).stickyKeyHashUnblocked(anyInt());
+
+ // when batch ends
+ tracker.endBatch();
+ // then unblocking call should be done
+ verify(unblockingHandler).stickyKeyHashUnblocked(-1);
+ }
+
+ @Test
+ public void clear_RemovesAllEntries() {
+ Consumer consumer = createMockConsumer("consumer1");
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class));
+ tracker.addEntry(consumer, 1);
+
+ tracker.clear();
+
+ assertNull(tracker.getEntry(1));
+ }
+
+ @Test
+ public void unblockingHandler_InvokesStickyKeyHashUnblocked() {
+ // given a tracker with unblocking handler
+ UnblockingHandler unblockingHandler = mock(UnblockingHandler.class);
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", unblockingHandler);
+
+ // when a hash is draining
+ Consumer consumer = createMockConsumer("consumer1");
+ tracker.addEntry(consumer, 1);
+ // aand hash gets blocked
+ Consumer consumer2 = createMockConsumer("consumer2");
+ tracker.shouldBlockStickyKeyHash(consumer2, 1);
+ // and hash gets unblocked
+ tracker.reduceRefCount(consumer, 1, false);
+
+ // then unblocking call should be done
+ verify(unblockingHandler).stickyKeyHashUnblocked(1);
+ }
+
+ @Test
+ public void unblockingHandler_DoesNotInvokeStickyKeyHashUnblockedWhenClosing() {
+ // given a tracker with unblocking handler
+ UnblockingHandler unblockingHandler = mock(UnblockingHandler.class);
+ DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", unblockingHandler);
+
+ // when a hash is draining
+ Consumer consumer = createMockConsumer("consumer1");
+ tracker.addEntry(consumer, 1);
+ // aand hash gets blocked
+ Consumer consumer2 = createMockConsumer("consumer2");
+ tracker.shouldBlockStickyKeyHash(consumer2, 1);
+ // and hash gets unblocked
+ tracker.reduceRefCount(consumer, 1, true);
+
+ // then unblocking call should be done
+ verify(unblockingHandler, never()).stickyKeyHashUnblocked(anyInt());
+ }
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java
new file mode 100644
index 0000000000000..42f5935ca88ff
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java
@@ -0,0 +1,196 @@
+/*
+ * 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 static org.apache.pulsar.broker.BrokerTestUtil.createMockConsumer;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+import java.util.ArrayList;
+import java.util.List;
+import org.testng.annotations.Test;
+
+public class PendingAcksMapTest {
+ @Test
+ public void addPendingAckIfAllowed_AddsAckWhenAllowed() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null);
+
+ boolean result = pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+
+ assertTrue(result);
+ assertTrue(pendingAcksMap.contains(1L, 1L));
+ }
+
+ @Test
+ public void addPendingAckIfAllowed_DoesNotAddAckWhenNotAllowed() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap.PendingAcksAddHandler addHandler = mock(PendingAcksMap.PendingAcksAddHandler.class);
+ when(addHandler.handleAdding(any(), anyLong(), anyLong(), anyInt())).thenReturn(false);
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> addHandler, () -> null);
+
+ boolean result = pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+
+ assertFalse(result);
+ assertFalse(pendingAcksMap.contains(1L, 1L));
+ }
+
+ @Test
+ public void addPendingAckIfAllowed_DoesNotAddAfterClosed() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null);
+ pendingAcksMap.forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> {});
+
+ boolean result = pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+
+ assertFalse(result);
+ assertFalse(pendingAcksMap.contains(1L, 1L));
+ }
+
+ @Test
+ public void forEach_ProcessesAllPendingAcks() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124);
+
+ List processedEntries = new ArrayList<>();
+ pendingAcksMap.forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> processedEntries.add(entryId));
+
+ assertEquals(processedEntries, List.of(1L, 2L));
+ }
+
+ @Test
+ public void forEachAndClose_ProcessesAndClearsAllPendingAcks() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124);
+
+ List processedEntries = new ArrayList<>();
+ pendingAcksMap.forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> processedEntries.add(entryId));
+
+ assertEquals(processedEntries, List.of(1L, 2L));
+ assertEquals(pendingAcksMap.size(), 0);
+ }
+
+ @Test
+ public void remove_RemovesPendingAck() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+
+ boolean result = pendingAcksMap.remove(1L, 1L);
+
+ assertTrue(result);
+ assertFalse(pendingAcksMap.contains(1L, 1L));
+ }
+
+ @Test
+ public void removeAllUpTo_RemovesAllPendingAcksUpToSpecifiedEntry() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124);
+ pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125);
+
+ pendingAcksMap.removeAllUpTo(1L, 2L);
+
+ assertFalse(pendingAcksMap.contains(1L, 1L));
+ assertFalse(pendingAcksMap.contains(1L, 2L));
+ assertTrue(pendingAcksMap.contains(2L, 1L));
+ }
+
+ @Test
+ public void removeAllUpTo_RemovesAllPendingAcksUpToSpecifiedEntryAcrossMultipleLedgers() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124);
+ pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125);
+ pendingAcksMap.addPendingAckIfAllowed(2L, 2L, 1, 126);
+ pendingAcksMap.addPendingAckIfAllowed(3L, 1L, 1, 127);
+
+ pendingAcksMap.removeAllUpTo(2L, 1L);
+
+ assertFalse(pendingAcksMap.contains(1L, 1L));
+ assertFalse(pendingAcksMap.contains(1L, 2L));
+ assertFalse(pendingAcksMap.contains(2L, 1L));
+ assertTrue(pendingAcksMap.contains(2L, 2L));
+ assertTrue(pendingAcksMap.contains(3L, 1L));
+ }
+
+ @Test
+ public void addPendingAckIfAllowed_InvokesAddHandler() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap.PendingAcksAddHandler addHandler = mock(PendingAcksMap.PendingAcksAddHandler.class);
+ when(addHandler.handleAdding(any(), anyLong(), anyLong(), anyInt())).thenReturn(true);
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> addHandler, () -> null);
+
+ pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+
+ verify(addHandler).handleAdding(consumer, 1L, 1L, 123);
+ }
+
+ @Test
+ public void remove_InvokesRemoveHandler() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap.PendingAcksRemoveHandler removeHandler = mock(PendingAcksMap.PendingAcksRemoveHandler.class);
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> removeHandler);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+
+ pendingAcksMap.remove(1L, 1L);
+
+ verify(removeHandler).handleRemoving(consumer, 1L, 1L, 123, false);
+ }
+
+ @Test
+ public void removeAllUpTo_InvokesRemoveHandlerForEachEntry() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap.PendingAcksRemoveHandler removeHandler = mock(PendingAcksMap.PendingAcksRemoveHandler.class);
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> removeHandler);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124);
+ pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125);
+
+ pendingAcksMap.removeAllUpTo(1L, 2L);
+
+ verify(removeHandler).handleRemoving(consumer, 1L, 1L, 123, false);
+ verify(removeHandler).handleRemoving(consumer, 1L, 2L, 124, false);
+ verify(removeHandler, never()).handleRemoving(consumer, 2L, 1L, 125, false);
+ }
+
+ @Test
+ public void size_ReturnsCorrectSize() {
+ Consumer consumer = createMockConsumer("consumer1");
+ PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123);
+ pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124);
+ pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125);
+
+ assertEquals(pendingAcksMap.size(), 3);
+ }
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
index 640cd2d37e399..9e85d9e3e91d1 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
@@ -18,7 +18,6 @@
*/
package org.apache.pulsar.broker.service;
-import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNotNull;
@@ -55,8 +54,6 @@
import org.apache.bookkeeper.mledger.impl.cache.EntryCache;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
-import org.apache.pulsar.broker.service.persistent.MessageRedeliveryController;
-import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.broker.service.schema.SchemaRegistry;
@@ -1721,90 +1718,6 @@ public void testMessageRedelivery() throws Exception {
producer.close();
}
- /**
- * Verify: 1. Broker should not replay already acknowledged messages 2. Dispatcher should not stuck while
- * dispatching new messages due to previous-replay of invalid/already-acked messages
- *
- * @throws Exception
- */
- @Test
- public void testMessageReplay() throws Exception {
-
- final String topicName = "persistent://prop/ns-abc/topic2";
- final String subName = "sub2";
-
- Message msg;
- int totalMessages = 10;
- int replayIndex = totalMessages / 2;
-
- Consumer consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subName)
- .subscriptionType(SubscriptionType.Shared).receiverQueueSize(1).subscribe();
- Producer producer = pulsarClient.newProducer()
- .topic(topicName)
- .enableBatching(false)
- .messageRoutingMode(MessageRoutingMode.SinglePartition)
- .create();
-
- PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get();
- assertNotNull(topicRef);
- PersistentSubscription subRef = topicRef.getSubscription(subName);
- PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) subRef
- .getDispatcher();
- Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class
- .getDeclaredField("redeliveryMessages");
- redeliveryMessagesField.setAccessible(true);
- MessageRedeliveryController redeliveryMessages = new MessageRedeliveryController(true);
-
- assertNotNull(subRef);
-
- // (1) Produce messages
- for (int i = 0; i < totalMessages; i++) {
- String message = "my-message-" + i;
- producer.send(message.getBytes());
- }
-
- MessageIdImpl firstAckedMsg = null;
- // (2) Consume and ack messages except first message
- for (int i = 0; i < totalMessages; i++) {
- msg = consumer.receive();
- consumer.acknowledge(msg);
- MessageIdImpl msgId = (MessageIdImpl) msg.getMessageId();
- if (i == 0) {
- firstAckedMsg = msgId;
- }
- if (i < replayIndex) {
- // (3) accumulate acked messages for replay
- redeliveryMessages.add(msgId.getLedgerId(), msgId.getEntryId());
- }
- }
-
- // (4) redelivery : should redeliver only unacked messages
- Thread.sleep(1000);
-
- redeliveryMessagesField.set(dispatcher, redeliveryMessages);
- // (a) redelivery with all acked-message should clear messageReply bucket
- dispatcher.redeliverUnacknowledgedMessages(dispatcher.getConsumers().get(0), DEFAULT_CONSUMER_EPOCH);
- Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> {
- return redeliveryMessages.isEmpty();
- });
- assertTrue(redeliveryMessages.isEmpty());
-
- // (b) fill messageReplyBucket with already acked entry again: and try to publish new msg and read it
- redeliveryMessages.add(firstAckedMsg.getLedgerId(), firstAckedMsg.getEntryId());
- redeliveryMessagesField.set(dispatcher, redeliveryMessages);
- // send new message
- final String testMsg = "testMsg";
- producer.send(testMsg.getBytes());
- // consumer should be able to receive only new message and not the
- dispatcher.consumerFlow(dispatcher.getConsumers().get(0), 1);
- msg = consumer.receive(1, TimeUnit.SECONDS);
- assertNotNull(msg);
- assertEquals(msg.getData(), testMsg.getBytes());
-
- consumer.close();
- producer.close();
- }
-
@Test
public void testCreateProducerWithSameName() throws Exception {
String topic = "persistent://prop/ns-abc/testCreateProducerWithSameName";
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 a03ed92b81590..052c5ceb5cdde 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
@@ -158,8 +158,8 @@ public void testSkipReadEntriesFromCloseCursor() throws Exception {
dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"),
null);
return null;
- }).when(cursor).asyncReadEntriesOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher),
- Mockito.any(), Mockito.any());
+ }).when(cursor).asyncReadEntriesWithSkipOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher),
+ Mockito.any(), Mockito.any(), Mockito.any());
dispatcher.readMoreEntries();
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 a0054f7e71425..4b29ead984e7a 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
@@ -37,8 +37,6 @@
import static org.mockito.Mockito.when;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
-import static org.testng.Assert.assertNull;
-import static org.testng.Assert.assertThrows;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
import com.google.common.collect.BoundType;
@@ -51,16 +49,15 @@
import io.netty.util.concurrent.GenericFutureListener;
import java.lang.reflect.Field;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Queue;
import java.util.Set;
+import java.util.TreeSet;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.bookkeeper.common.util.OrderedExecutor;
import org.apache.bookkeeper.mledger.Entry;
@@ -73,6 +70,7 @@
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.EntryAndMetadata;
import org.apache.pulsar.broker.service.EntryBatchIndexesAcks;
import org.apache.pulsar.broker.service.EntryBatchSizes;
import org.apache.pulsar.broker.service.RedeliveryTracker;
@@ -85,8 +83,6 @@
import org.apache.pulsar.common.policies.data.HierarchyTopicPolicies;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.Markers;
-import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet;
-import org.apache.pulsar.common.util.collections.LongPairRangeSet;
import org.awaitility.Awaitility;
import org.mockito.ArgumentCaptor;
import org.testng.Assert;
@@ -123,9 +119,10 @@ public void setup() throws Exception {
doReturn(true).when(configMock).isSubscriptionRedeliveryTrackerEnabled();
doReturn(100).when(configMock).getDispatcherMaxReadBatchSize();
doReturn(true).when(configMock).isSubscriptionKeySharedUseConsistentHashing();
- doReturn(1).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints();
+ doReturn(20).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints();
doReturn(false).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread();
doReturn(false).when(configMock).isAllowOverrideEntryFilters();
+ doReturn(false).when(configMock).isDispatchThrottlingOnNonBacklogConsumerEnabled();
doAnswer(invocation -> retryBackoffInitialTimeInMs).when(configMock).getDispatcherRetryBackoffInitialTimeInMs();
doAnswer(invocation -> retryBackoffMaxTimeInMs).when(configMock).getDispatcherRetryBackoffMaxTimeInMs();
pulsarMock = mock(PulsarService.class);
@@ -223,6 +220,8 @@ protected static Consumer createMockConsumer() {
TransportCnx transportCnx = mock(TransportCnx.class);
doReturn(transportCnx).when(consumerMock).cnx();
doReturn(true).when(transportCnx).isActive();
+ doReturn(100).when(consumerMock).getMaxUnackedMessages();
+ doReturn(1).when(consumerMock).getAvgMessagesPerEntry();
return consumerMock;
}
@@ -321,13 +320,16 @@ public void testSendMessage() {
@Test
public void testSkipRedeliverTemporally() {
final Consumer slowConsumerMock = createMockConsumer();
+ AtomicInteger slowConsumerPermits = new AtomicInteger(0);
+ doAnswer(invocation -> slowConsumerPermits.get()).when(slowConsumerMock).getAvailablePermits();
+
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")));
+ redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key123")));
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")));
+ readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key123")));
+ readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key222")));
try {
Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumers.class.getDeclaredField("totalAvailablePermits");
@@ -348,9 +350,6 @@ public void testSkipRedeliverTemporally() {
// Create 2Consumers
try {
doReturn("consumer2").when(slowConsumerMock).consumerName();
- when(slowConsumerMock.getAvailablePermits())
- .thenReturn(0)
- .thenReturn(1);
doReturn(true).when(slowConsumerMock).isWritable();
doReturn(slowChannelMock).when(slowConsumerMock).sendMessages(
anyList(),
@@ -375,13 +374,12 @@ public void testSkipRedeliverTemporally() {
// and then stop to dispatch to slowConsumer
persistentDispatcher.readEntriesComplete(redeliverEntries,
PersistentDispatcherMultipleConsumers.ReadType.Replay);
-
verify(consumerMock, times(1)).sendMessages(
argThat(arg -> {
assertEquals(arg.size(), 1);
Entry entry = arg.get(0);
assertEquals(entry.getLedgerId(), 1);
- assertEquals(entry.getEntryId(), 3);
+ assertEquals(entry.getEntryId(), 1);
return true;
}),
any(EntryBatchSizes.class),
@@ -408,25 +406,9 @@ public void testMessageRedelivery() throws Exception {
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 AtomicInteger remainingEntriesNum = new AtomicInteger(0);
final Consumer consumer1 = createMockConsumer();
doReturn("consumer1").when(consumer1).consumerName();
@@ -434,8 +416,7 @@ public void testMessageRedelivery() throws Exception {
when(consumer1.getAvailablePermits()).thenReturn(0).thenReturn(10);
doReturn(true).when(consumer1).isWritable();
doAnswer(invocationOnMock -> {
- @SuppressWarnings("unchecked")
- List entries = (List) invocationOnMock.getArgument(0);
+ List entries = invocationOnMock.getArgument(0);
for (Entry entry : entries) {
remainingEntriesNum.decrementAndGet();
actualEntriesToConsumer1.add(entry.getPosition());
@@ -449,8 +430,7 @@ public void testMessageRedelivery() throws Exception {
when(consumer2.getAvailablePermits()).thenReturn(10);
doReturn(true).when(consumer2).isWritable();
doAnswer(invocationOnMock -> {
- @SuppressWarnings("unchecked")
- List entries = (List) invocationOnMock.getArgument(0);
+ List entries = invocationOnMock.getArgument(0);
for (Entry entry : entries) {
remainingEntriesNum.decrementAndGet();
actualEntriesToConsumer2.add(entry.getPosition());
@@ -467,44 +447,64 @@ public void testMessageRedelivery() throws Exception {
totalAvailablePermitsField.setAccessible(true);
totalAvailablePermitsField.set(persistentDispatcher, 1000);
- final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class
- .getDeclaredField("redeliveryMessages");
- redeliveryMessagesField.setAccessible(true);
- MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField
- .get(persistentDispatcher);
- redeliveryMessages.add(allEntries.get(0).getLedgerId(), allEntries.get(0).getEntryId(),
- getStickyKeyHash(allEntries.get(0))); // message1
- redeliveryMessages.add(allEntries.get(1).getLedgerId(), allEntries.get(1).getEntryId(),
- getStickyKeyHash(allEntries.get(1))); // message2
+ StickyKeyConsumerSelector selector = persistentDispatcher.getSelector();
+
+ String keyForConsumer1 = generateKeyForConsumer(selector, consumer1);
+ String keyForConsumer2 = generateKeyForConsumer(selector, consumer2);
+
+ // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2
+ final List allEntries = new ArrayList<>();
+ allEntries.add(EntryAndMetadata.create(EntryImpl.create(1, 1, createMessage("message1", 1, keyForConsumer1))));
+ allEntries.add(EntryAndMetadata.create(EntryImpl.create(1, 2, createMessage("message2", 2, keyForConsumer1))));
+ allEntries.add(EntryAndMetadata.create(EntryImpl.create(1, 3, createMessage("message3", 3, keyForConsumer2))));
+ allEntries.forEach(entry -> {
+ EntryImpl entryImpl = (EntryImpl) ((EntryAndMetadata) entry).unwrap();
+ entryImpl.retain();
+ // initialize sticky key hash
+ persistentDispatcher.getStickyKeyHash(entry);
+ });
+ remainingEntriesNum.set(allEntries.size());
+
+ final List redeliverEntries = new ArrayList<>();
+ redeliverEntries.add(allEntries.get(0)); // message1
+ final List readEntries = new ArrayList<>();
+ readEntries.add(allEntries.get(2)); // message3
+
+ expectedEntriesToConsumer1.add(allEntries.get(0).getPosition());
+ expectedEntriesToConsumer1.add(allEntries.get(1).getPosition());
+ expectedEntriesToConsumer2.add(allEntries.get(2).getPosition());
// Mock Cursor#asyncReplayEntries
doAnswer(invocationOnMock -> {
- @SuppressWarnings("unchecked")
- Set positions = (Set) invocationOnMock.getArgument(0);
- List entries = allEntries.stream().filter(entry -> positions.contains(entry.getPosition()))
+ Set positionsArg = invocationOnMock.getArgument(0);
+ Set positions = new TreeSet<>(positionsArg);
+ Set alreadyReceived = new TreeSet<>();
+ alreadyReceived.addAll(actualEntriesToConsumer1);
+ alreadyReceived.addAll(actualEntriesToConsumer2);
+ List entries = allEntries.stream().filter(entry -> positions.contains(entry.getPosition())
+ && !alreadyReceived.contains(entry.getPosition()))
.collect(Collectors.toList());
- if (!entries.isEmpty()) {
- ((PersistentStickyKeyDispatcherMultipleConsumers) invocationOnMock.getArgument(1))
- .readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay);
- }
- return Collections.emptySet();
+ PersistentStickyKeyDispatcherMultipleConsumers dispatcher = invocationOnMock.getArgument(1);
+ dispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay);
+ return alreadyReceived;
}).when(cursorMock).asyncReplayEntries(anySet(), any(PersistentStickyKeyDispatcherMultipleConsumers.class),
eq(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay), anyBoolean());
// Mock Cursor#asyncReadEntriesOrWait
- AtomicBoolean asyncReadEntriesOrWaitCalled = new AtomicBoolean();
doAnswer(invocationOnMock -> {
- if (asyncReadEntriesOrWaitCalled.compareAndSet(false, true)) {
- ((PersistentStickyKeyDispatcherMultipleConsumers) invocationOnMock.getArgument(2))
- .readEntriesComplete(readEntries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal);
- } else {
- ((PersistentStickyKeyDispatcherMultipleConsumers) invocationOnMock.getArgument(2))
- .readEntriesComplete(Collections.emptyList(), PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal);
- }
+ int maxEntries = invocationOnMock.getArgument(0);
+ Set alreadyReceived = new TreeSet<>();
+ alreadyReceived.addAll(actualEntriesToConsumer1);
+ alreadyReceived.addAll(actualEntriesToConsumer2);
+ List entries = allEntries.stream()
+ .filter(entry -> !alreadyReceived.contains(entry.getPosition()))
+ .limit(maxEntries).collect(Collectors.toList());
+ PersistentStickyKeyDispatcherMultipleConsumers dispatcher = invocationOnMock.getArgument(2);
+ dispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal);
return null;
- }).when(cursorMock).asyncReadEntriesOrWait(anyInt(), anyLong(),
+ }).when(cursorMock).asyncReadEntriesWithSkipOrWait(anyInt(), anyLong(),
any(PersistentStickyKeyDispatcherMultipleConsumers.class),
- eq(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal), any());
+ eq(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal), any(), any());
// (1) Run sendMessagesToConsumers
// (2) Attempts to send message1 to consumer1 but skipped because availablePermits is 0
@@ -512,6 +512,11 @@ public void testMessageRedelivery() throws Exception {
// (4) Run readMoreEntries internally
// (5) Run sendMessagesToConsumers internally
// (6) Attempts to send message3 to consumer2 but skipped because redeliveryMessages contains message2
+ redeliverEntries.forEach(entry -> {
+ EntryImpl entryImpl = (EntryImpl) ((EntryAndMetadata) entry).unwrap();
+ entryImpl.retain();
+ persistentDispatcher.addEntryToReplay(entry);
+ });
persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay,
redeliverEntries, true);
while (remainingEntriesNum.get() > 0) {
@@ -525,313 +530,16 @@ public void testMessageRedelivery() throws Exception {
allEntries.forEach(entry -> entry.release());
}
- @DataProvider(name = "initializeLastSentPosition")
- private Object[][] initialLastSentPositionProvider() {
- return new Object[][] { { false }, { true } };
- }
-
- @Test(dataProvider = "initializeLastSentPosition")
- public void testLastSentPositionAndIndividuallySentPositions(final boolean initializeLastSentPosition) throws Exception {
- final Position initialLastSentPosition = PositionFactory.create(1, 10);
- final LongPairRangeSet expectedIndividuallySentPositions
- = new ConcurrentOpenLongPairRangeSet<>(4096, PositionFactory::create);
-
- final Field lastSentPositionField = PersistentStickyKeyDispatcherMultipleConsumers.class
- .getDeclaredField("lastSentPosition");
- lastSentPositionField.setAccessible(true);
- final LongPairRangeSet individuallySentPositions = persistentDispatcher.getIndividuallySentPositionsField();
- final Supplier clearPosition = () -> {
- try {
- lastSentPositionField.set(persistentDispatcher, initializeLastSentPosition ? initialLastSentPosition : null);
- individuallySentPositions.clear();
- expectedIndividuallySentPositions.clear();
- } catch (Throwable e) {
- return e;
+ private String generateKeyForConsumer(StickyKeyConsumerSelector selector, Consumer consumer) {
+ int i = 0;
+ while (!Thread.currentThread().isInterrupted()) {
+ String key = "key" + i++;
+ Consumer selectedConsumer = selector.select(key.getBytes(UTF_8));
+ if (selectedConsumer == consumer) {
+ return key;
}
- return null;
- };
- if (!initializeLastSentPosition) {
- doReturn(initialLastSentPosition).when(cursorMock).getMarkDeletedPosition();
- doAnswer(invocationOnMock -> {
- // skip copy operation
- return initialLastSentPosition;
- }).when(cursorMock).processIndividuallyDeletedMessagesAndGetMarkDeletedPosition(any());
}
-
- // Assume the range sequence is [1:0, 1:19], [2:0, 2:19], ..., [10:0, 10:19]
- doAnswer((invocationOnMock -> {
- final Position position = invocationOnMock.getArgument(0);
- if (position.getEntryId() > 0) {
- return PositionFactory.create(position.getLedgerId(), position.getEntryId() - 1);
- } else if (position.getLedgerId() > 0) {
- return PositionFactory.create(position.getLedgerId() - 1, 19);
- } else {
- throw new NullPointerException();
- }
- })).when(ledgerMock).getPreviousPosition(any(Position.class));
- doAnswer((invocationOnMock -> {
- final Position position = invocationOnMock.getArgument(0);
- if (position.getEntryId() < 19) {
- return PositionFactory.create(position.getLedgerId(), position.getEntryId() + 1);
- } else {
- return PositionFactory.create(position.getLedgerId() + 1, 0);
- }
- })).when(ledgerMock).getNextValidPosition(any(Position.class));
- doReturn(PositionFactory.create(10, 19)).when(ledgerMock).getLastConfirmedEntry();
- doAnswer((invocationOnMock -> {
- final Range range = invocationOnMock.getArgument(0);
- Position fromPosition = range.lowerEndpoint();
- boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED;
- Position toPosition = range.upperEndpoint();
- boolean toIncluded = range.upperBoundType() == BoundType.CLOSED;
-
- if (fromPosition.getLedgerId() == toPosition.getLedgerId()) {
- // If the 2 positions are in the same ledger
- long count = toPosition.getEntryId() - fromPosition.getEntryId() - 1;
- count += fromIncluded ? 1 : 0;
- count += toIncluded ? 1 : 0;
- return count;
- } else {
- long count = 0;
- // If the from & to are pointing to different ledgers, then we need to :
- // 1. Add the entries in the ledger pointed by toPosition
- count += toPosition.getEntryId();
- count += toIncluded ? 1 : 0;
-
- // 2. Add the entries in the ledger pointed by fromPosition
- count += 20 - (fromPosition.getEntryId() + 1);
- count += fromIncluded ? 1 : 0;
-
- // 3. Add the whole ledgers entries in between
- for (long i = fromPosition.getLedgerId() + 1; i < toPosition.getLedgerId(); i++) {
- count += 20;
- }
-
- return count;
- }
- })).when(ledgerMock).getNumberOfEntries(any());
- assertEquals(ledgerMock.getNextValidPosition(PositionFactory.create(1, 0)), PositionFactory.create(1, 1));
- assertEquals(ledgerMock.getNextValidPosition(PositionFactory.create(1, 19)), PositionFactory.create(2, 0));
- assertEquals(ledgerMock.getPreviousPosition(PositionFactory.create(2, 0)), PositionFactory.create(1, 19));
- assertThrows(NullPointerException.class, () -> ledgerMock.getPreviousPosition(PositionFactory.create(0, 0)));
- assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed(
- PositionFactory.create(1, 0), PositionFactory.create(1, 0))), 0);
- assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed(
- PositionFactory.create(1, -1), PositionFactory.create(1, 9))), 10);
- assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed(
- PositionFactory.create(1, 19), PositionFactory.create(2, -1))), 0);
- assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed(
- PositionFactory.create(1, 19), PositionFactory.create(2, 9))), 10);
- assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed(
- PositionFactory.create(1, -1), PositionFactory.create(3, 19))), 60);
-
- // Add a consumer
- final Consumer consumer1 = createMockConsumer();
- doReturn("consumer1").when(consumer1).consumerName();
- when(consumer1.getAvailablePermits()).thenReturn(1000);
- doReturn(true).when(consumer1).isWritable();
- doReturn(channelMock).when(consumer1).sendMessages(anyList(), any(EntryBatchSizes.class),
- any(EntryBatchIndexesAcks.class), anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class));
- persistentDispatcher.addConsumer(consumer1);
-
- /*
- On single ledger
- */
-
- // Expected individuallySentPositions (isp): [(1:-1, 1:8]] (init) -> [(1:-1, 1:9]] (update) -> [] (remove)
- // Expected lastSentPosition (lsp): 1:10 (init) -> 1:10 (remove)
- // upper bound and the new entry are less than initial last sent position
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, -1, 1, 8);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 9, createMessage("test", 1))), true);
- assertTrue(individuallySentPositions.isEmpty());
- assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString());
-
- // isp: [(1:-1, 1:9]] -> [(1:-1, 1:10]] -> []
- // lsp: 1:10 -> 1:10
- // upper bound is less than initial last sent position
- // upper bound and the new entry are less than or equal to initial last sent position
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, -1, 1, 9);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 10, createMessage("test", 1))), true);
- assertTrue(individuallySentPositions.isEmpty());
- assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString());
-
- // isp: [(1:-1, 1:2], (1:3, 1:4], (1:5, 1:6]] -> [(1:-1, 1:2], (1:3, 1:4], (1:5, 1:6], (1:9, 1:10]] -> []
- // lsp: 1:10 -> 1:10
- // upper bound and the new entry are less than or equal to initial last sent position
- // individually sent positions has multiple ranges
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, -1, 1, 2);
- individuallySentPositions.addOpenClosed(1, 3, 1, 4);
- individuallySentPositions.addOpenClosed(1, 5, 1, 6);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 10, createMessage("test", 1))), true);
- assertTrue(individuallySentPositions.isEmpty());
- assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString());
-
- // isp: [(1:-1, 1:10]] -> [(1:-1, 1:11]] -> []
- // lsp: 1:10 -> 1:11
- // upper bound is less than or equal to initial last sent position
- // the new entry is next position of initial last sent position
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, -1, 1, 10);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true);
- assertTrue(individuallySentPositions.isEmpty());
- assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 11).toString());
-
- // isp: [(1:-1, 1:9]] -> [(1:-1, 1:9], (1:10, 1:11]] -> []
- // lsp: 1:10 -> 1:11
- // upper bound is less than initial last sent position
- // the new entry is next position of initial last sent position
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, -1, 1, 9);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true);
- assertTrue(individuallySentPositions.isEmpty());
- assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 11).toString());
-
- // isp: [(1:11, 1:15]] -> [(1:10, 1:15]] -> []
- // lsp: 1:10 -> 1:15
- // upper bound is greater than initial last sent position
- // the range doesn't contain next position of initial last sent position
- // the new entry is next position of initial last sent position
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, 11, 1, 15);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true);
- assertTrue(individuallySentPositions.isEmpty());
- assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 15).toString());
-
- // isp: [(1:11, 1:15]] -> [(1:10, 1:16]] -> []
- // lsp: 1:10 -> 1:16
- // upper bound is greater than initial last sent position
- // the range doesn't contain next position of initial last sent position
- // the new entries contain next position of initial last sent position
- // first of the new entries is less than initial last sent position
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, 11, 1, 15);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 9, createMessage("test", 1)),
- EntryImpl.create(1, 11, createMessage("test", 2)),
- EntryImpl.create(1, 16, createMessage("test", 3))), true);
- assertTrue(individuallySentPositions.isEmpty());
- assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 16).toString());
-
- // isp: [(1:11, 1:15]] -> [(1:11, 1:15]] -> [(1:11, 1:15]]
- // lsp: 1:10 -> 1:10
- // upper bound is greater than initial last sent position
- // the range doesn't contain next position of initial last sent position
- // the new entry isn't next position of initial last sent position
- // the range contains the new entry
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, 11, 1, 15);
- expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 15);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 15, createMessage("test", 1))), true);
- assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString());
- assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString());
-
- // isp: [(1:11, 1:15]] -> [(1:11, 1:16]] -> [(1:11, 1:16]]
- // lsp: 1:10 -> 1:10
- // upper bound is greater than initial last sent position
- // the range doesn't contain next position of initial last sent position
- // the new entry isn't next position of initial last sent position
- // the range doesn't contain the new entry
- // the new entry is next position of upper bound
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, 11, 1, 15);
- expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 16);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 16, createMessage("test", 1))), true);
- assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString());
- assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString());
-
- // isp: [(1:11, 1:15]] -> [(1:11, 1:15], (1:16, 1:17]] -> [(1:11, 1:15], (1:16, 1:17]]
- // lsp: 1:10 -> 1:10
- // upper bound is greater than initial last sent position
- // the range doesn't contain next position of initial last sent position
- // the new entry isn't next position of initial last sent position
- // the range doesn't contain the new entry
- // the new entry isn't next position of upper bound
- // the new entry is same ledger
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, 11, 1, 15);
- expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 15);
- expectedIndividuallySentPositions.addOpenClosed(1, 16, 1, 17);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 17, createMessage("test", 1))), true);
- assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString());
- assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString());
-
- /*
- On multiple contiguous ledgers
- */
-
- // isp: [(1:11, 1:18]] -> [(1:11, 1:18], (2:-1, 2:0]] -> [(1:11, 1:18], (2:-1, 2:0]]
- // lsp: 1:10 -> 1:10
- // upper bound is greater than initial last sent position
- // the range doesn't contain next position of initial last sent position
- // the new entry isn't next position of initial last sent position
- // the range doesn't contain the new entry
- // the new entry isn't next position of upper bound
- // the new entry isn't same ledger
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, 11, 1, 18);
- expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 18);
- expectedIndividuallySentPositions.addOpenClosed(2, -1, 2, 0);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(2, 0, createMessage("test", 1))), true);
- assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString());
- assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString());
-
- // isp: [(1:11, 1:19], (2:-1, 2:0]] -> [(1:10, 1:19], (2:-1, 2:0]] -> []
- // lsp: 1:10 -> 2:0
- // upper bound is greater than initial last sent position
- // the range doesn't contain next position of initial last sent position
- // the new entry is next position of initial last sent position
- // the new entry isn't same ledger
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, 11, 1, 19);
- individuallySentPositions.addOpenClosed(2, -1, 2, 0);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true);
- assertTrue(individuallySentPositions.isEmpty());
- assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(2, 0).toString());
-
- // isp: [(1:11, 1:19], (2:-1, 2:19], (3:-1, 3:0]] -> [(1:10, 1:19], (2:-1, 2:19], (3:-1, 3:0]] -> []
- // lsp: 1:10 -> 3:0
- // upper bound is greater than initial last sent position
- // the range doesn't contain next position of initial last sent position
- // the new entry is next position of initial last sent position
- // the new entry isn't same ledger
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, 11, 1, 19);
- individuallySentPositions.addOpenClosed(2, -1, 2, 19);
- individuallySentPositions.addOpenClosed(3, -1, 3, 0);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true);
- assertTrue(individuallySentPositions.isEmpty());
- assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(3, 0).toString());
-
- // isp: [(1:11, 1:19], (2:-1, 2:0]] -> [(1:11, 1:19], (2:-1, 2:1]] -> [(1:11, 1:19], (2:-1, 2:1]]
- // lsp: 1:10 -> 1:10
- // upper bound is greater than initial last sent position
- // the range doesn't contain next position of initial last sent position
- // the new entry isn't next position of initial last sent position
- // the new entry isn't same ledger
- assertNull(clearPosition.get());
- individuallySentPositions.addOpenClosed(1, 11, 1, 19);
- individuallySentPositions.addOpenClosed(2, -1, 2, 0);
- expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 19);
- expectedIndividuallySentPositions.addOpenClosed(2, -1, 2, 1);
- persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal,
- Arrays.asList(EntryImpl.create(2, 1, createMessage("test", 1))), true);
- assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString());
- assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString());
+ return null;
}
@DataProvider(name = "testBackoffDelayWhenNoMessagesDispatched")
@@ -872,7 +580,7 @@ protected void reScheduleReadInMs(long readAfterMs) {
// call "readEntriesComplete" directly to test the retry behavior
List entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1)));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
Awaitility.await().untilAsserted(() -> {
assertEquals(retryDelays.size(), 1);
assertEquals(retryDelays.get(0), 10, "Initial retry delay should be 10ms");
@@ -880,7 +588,7 @@ protected void reScheduleReadInMs(long readAfterMs) {
);
// test the second retry delay
entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1)));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
Awaitility.await().untilAsserted(() -> {
assertEquals(retryDelays.size(), 2);
double delay = retryDelays.get(1);
@@ -890,7 +598,7 @@ protected void reScheduleReadInMs(long readAfterMs) {
// verify the max retry delay
for (int i = 0; i < 100; i++) {
entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1)));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
}
Awaitility.await().untilAsserted(() -> {
assertEquals(retryDelays.size(), 102);
@@ -901,14 +609,14 @@ protected void reScheduleReadInMs(long readAfterMs) {
// unblock to check that the retry delay is reset
consumerMockAvailablePermits.set(1000);
entries = List.of(EntryImpl.create(1, 2, createMessage("message2", 1, "key2")));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
// wait that the possibly async handling has completed
Awaitility.await().untilAsserted(() -> assertFalse(dispatcher.isSendInProgress()));
// now block again to check the next retry delay so verify it was reset
consumerMockAvailablePermits.set(0);
entries = List.of(EntryImpl.create(1, 3, createMessage("message3", 1, "key3")));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
Awaitility.await().untilAsserted(() -> {
assertEquals(retryDelays.size(), 103);
assertEquals(retryDelays.get(0), 10, "Resetted retry delay should be 10ms");
@@ -955,7 +663,7 @@ protected void reScheduleReadInMs(long readAfterMs) {
// call "readEntriesComplete" directly to test the retry behavior
List entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1)));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
Awaitility.await().untilAsserted(() -> {
assertEquals(retryDelays.size(), 1);
assertEquals(retryDelays.get(0), 0, "Initial retry delay should be 0ms");
@@ -963,7 +671,7 @@ protected void reScheduleReadInMs(long readAfterMs) {
);
// test the second retry delay
entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1)));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
Awaitility.await().untilAsserted(() -> {
assertEquals(retryDelays.size(), 2);
double delay = retryDelays.get(1);
@@ -973,7 +681,7 @@ protected void reScheduleReadInMs(long readAfterMs) {
// verify the max retry delay
for (int i = 0; i < 100; i++) {
entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1)));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
}
Awaitility.await().untilAsserted(() -> {
assertEquals(retryDelays.size(), 102);
@@ -984,14 +692,14 @@ protected void reScheduleReadInMs(long readAfterMs) {
// unblock to check that the retry delay is reset
consumerMockAvailablePermits.set(1000);
entries = List.of(EntryImpl.create(1, 2, createMessage("message2", 1, "key2")));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
// wait that the possibly async handling has completed
Awaitility.await().untilAsserted(() -> assertFalse(dispatcher.isSendInProgress()));
// now block again to check the next retry delay so verify it was reset
consumerMockAvailablePermits.set(0);
entries = List.of(EntryImpl.create(1, 3, createMessage("message3", 1, "key3")));
- dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal);
+ dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal);
Awaitility.await().untilAsserted(() -> {
assertEquals(retryDelays.size(), 103);
assertEquals(retryDelays.get(0), 0, "Resetted retry delay should be 0ms");
@@ -1092,9 +800,4 @@ private ByteBuf createMessage(String message, int sequenceId, String key) {
.setPublishTime(System.currentTimeMillis());
return serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, Unpooled.copiedBuffer(message.getBytes(UTF_8)));
}
-
- private int getStickyKeyHash(Entry entry) {
- byte[] stickyKey = Commands.peekStickyKey(entry.getDataBuffer(), topicName, subscriptionName);
- return StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey);
- }
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java
new file mode 100644
index 0000000000000..cf404e38b66d3
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.mockito.Mockito.any;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BooleanSupplier;
+import java.util.function.LongSupplier;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+public class RescheduleReadHandlerTest {
+ private LongSupplier readIntervalMsSupplier;
+ private ScheduledExecutorService executor;
+ private Runnable cancelPendingRead;
+ private Runnable rescheduleReadImmediately;
+ private BooleanSupplier hasPendingReadRequestThatMightWait;
+ private LongSupplier readOpCounterSupplier;
+ private BooleanSupplier hasEntriesInReplayQueue;
+ private RescheduleReadHandler rescheduleReadHandler;
+
+ @BeforeMethod
+ public void setUp() {
+ readIntervalMsSupplier = mock(LongSupplier.class);
+ executor = mock(ScheduledExecutorService.class);
+ cancelPendingRead = mock(Runnable.class);
+ rescheduleReadImmediately = mock(Runnable.class);
+ hasPendingReadRequestThatMightWait = mock(BooleanSupplier.class);
+ readOpCounterSupplier = mock(LongSupplier.class);
+ hasEntriesInReplayQueue = mock(BooleanSupplier.class);
+ rescheduleReadHandler = new RescheduleReadHandler(readIntervalMsSupplier, executor, cancelPendingRead,
+ () -> rescheduleReadImmediately.run(), hasPendingReadRequestThatMightWait, readOpCounterSupplier, hasEntriesInReplayQueue);
+ }
+
+ @Test
+ public void rescheduleReadImmediately() {
+ when(readIntervalMsSupplier.getAsLong()).thenReturn(0L);
+
+ rescheduleReadHandler.rescheduleRead();
+
+ verify(rescheduleReadImmediately).run();
+ verify(executor, never()).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class));
+ }
+
+ @Test
+ public void rescheduleReadWithDelay() {
+ when(readIntervalMsSupplier.getAsLong()).thenReturn(100L);
+
+ rescheduleReadHandler.rescheduleRead();
+
+ verify(rescheduleReadImmediately, never()).run();
+ verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS));
+ }
+
+ @Test
+ public void rescheduleReadWithDelayAndCancelPendingRead() {
+ when(readIntervalMsSupplier.getAsLong()).thenReturn(100L);
+ when(hasPendingReadRequestThatMightWait.getAsBoolean()).thenReturn(true);
+ when(readOpCounterSupplier.getAsLong()).thenReturn(5L);
+ when(hasEntriesInReplayQueue.getAsBoolean()).thenReturn(true);
+ doAnswer(invocation -> {
+ Runnable runnable = invocation.getArgument(0);
+ runnable.run();
+ return null;
+ }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class));
+
+ rescheduleReadHandler.rescheduleRead();
+
+ verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS));
+ verify(rescheduleReadImmediately).run();
+ verify(cancelPendingRead).run();
+ }
+
+ @Test
+ public void rescheduleReadWithDelayAndDontCancelPendingReadIfNoEntriesInReplayQueue() {
+ when(readIntervalMsSupplier.getAsLong()).thenReturn(100L);
+ when(hasPendingReadRequestThatMightWait.getAsBoolean()).thenReturn(true);
+ when(readOpCounterSupplier.getAsLong()).thenReturn(5L);
+ when(hasEntriesInReplayQueue.getAsBoolean()).thenReturn(false);
+ doAnswer(invocation -> {
+ Runnable runnable = invocation.getArgument(0);
+ runnable.run();
+ return null;
+ }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class));
+
+ rescheduleReadHandler.rescheduleRead();
+
+ verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS));
+ verify(rescheduleReadImmediately).run();
+ verify(cancelPendingRead, never()).run();
+ }
+
+ @Test
+ public void rescheduleReadBatching() {
+ when(readOpCounterSupplier.getAsLong()).thenReturn(5L);
+ when(readIntervalMsSupplier.getAsLong()).thenReturn(100L);
+ AtomicReference scheduledRunnable = new AtomicReference<>();
+ doAnswer(invocation -> {
+ Runnable runnable = invocation.getArgument(0);
+ if (!scheduledRunnable.compareAndSet(null, runnable)) {
+ runnable.run();
+ }
+ return null;
+ }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class));
+ // 3 rescheduleRead calls
+ rescheduleReadHandler.rescheduleRead();
+ rescheduleReadHandler.rescheduleRead();
+ rescheduleReadHandler.rescheduleRead();
+ // scheduled task runs
+ scheduledRunnable.get().run();
+ // verify that rescheduleReadImmediately is called only once
+ verify(rescheduleReadImmediately, times(1)).run();
+ }
+
+ @Test
+ public void rescheduleReadWithoutCancelPendingReadWhenReadOpCounterIncrements() {
+ // given
+ when(readIntervalMsSupplier.getAsLong()).thenReturn(100L);
+ when(hasPendingReadRequestThatMightWait.getAsBoolean()).thenReturn(true);
+ when(readOpCounterSupplier.getAsLong()).thenReturn(5L).thenReturn(6L);
+ when(hasEntriesInReplayQueue.getAsBoolean()).thenReturn(true);
+ doAnswer(invocation -> {
+ Runnable runnable = invocation.getArgument(0);
+ runnable.run();
+ return null;
+ }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class));
+ // when rescheduleRead is called
+ rescheduleReadHandler.rescheduleRead();
+ // then verify calls
+ verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS));
+ verify(rescheduleReadImmediately).run();
+ // verify that cancelPendingRead is not called
+ verify(cancelPendingRead, never()).run();
+ }
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java
index bc4cb73e5b6fe..4a8e7077395ed 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java
@@ -38,12 +38,10 @@
import org.apache.pulsar.broker.service.plugin.EntryFilter;
import org.apache.pulsar.broker.service.plugin.EntryFilterTest;
import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader;
-import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerConsumerBase;
-import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.common.naming.TopicName;
@@ -83,48 +81,6 @@ protected void cleanup() throws Exception {
super.internalCleanup();
}
- @Test
- public void testConsumersAfterMarkDelete() throws PulsarClientException, PulsarAdminException {
- final String topicName = "persistent://my-property/my-ns/testConsumersAfterMarkDelete-"
- + UUID.randomUUID();
- final String subName = "my-sub";
-
- Consumer consumer1 = pulsarClient.newConsumer()
- .topic(topicName)
- .receiverQueueSize(10)
- .subscriptionName(subName)
- .subscriptionType(SubscriptionType.Key_Shared)
- .subscribe();
-
- Producer producer = pulsarClient.newProducer()
- .topic(topicName)
- .create();
-
- final int messages = 100;
- for (int i = 0; i < messages; i++) {
- producer.send(String.valueOf(i).getBytes());
- }
-
- // Receive by do not ack the message, so that the next consumer can added to the recentJoinedConsumer of the dispatcher.
- consumer1.receive();
-
- Consumer consumer2 = pulsarClient.newConsumer()
- .topic(topicName)
- .receiverQueueSize(10)
- .subscriptionName(subName)
- .subscriptionType(SubscriptionType.Key_Shared)
- .subscribe();
-
- TopicStats stats = admin.topics().getStats(topicName);
- Assert.assertEquals(stats.getSubscriptions().size(), 1);
- Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue()
- .getConsumersAfterMarkDeletePosition().size(), 1);
-
- consumer1.close();
- consumer2.close();
- producer.close();
- }
-
@Test
public void testNonContiguousDeletedMessagesRanges() throws Exception {
final String topicName = "persistent://my-property/my-ns/testNonContiguousDeletedMessagesRanges-"
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 c08c37b413f4f..2b16647f5590c 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
@@ -19,11 +19,9 @@
package org.apache.pulsar.client.api;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName;
import static org.apache.pulsar.broker.BrokerTestUtil.receiveMessages;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.spy;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNotNull;
@@ -32,7 +30,6 @@
import static org.testng.Assert.fail;
import com.google.common.collect.Lists;
import java.io.IOException;
-import java.lang.reflect.Field;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.time.Duration;
@@ -41,7 +38,6 @@
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@@ -51,6 +47,7 @@
import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
@@ -59,33 +56,30 @@
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Supplier;
+import java.util.function.BiFunction;
import java.util.stream.Collectors;
import lombok.Cleanup;
+import lombok.SneakyThrows;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.PositionFactory;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
-import org.apache.pulsar.broker.BrokerTestUtil;
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.ServiceConfiguration;
+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.Topic;
import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers;
-import org.apache.pulsar.broker.service.persistent.MessageRedeliveryController;
-import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.impl.ConsumerImpl;
-import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.common.api.proto.KeySharedMode;
import org.apache.pulsar.common.naming.TopicDomain;
import org.apache.pulsar.common.naming.TopicName;
-import org.apache.pulsar.common.policies.data.RetentionPolicies;
import org.apache.pulsar.common.schema.KeyValue;
-import org.apache.pulsar.common.util.Murmur3_32Hash;
-import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet;
-import org.apache.pulsar.common.util.collections.LongPairRangeSet;
+import org.assertj.core.api.Assertions;
import org.awaitility.Awaitility;
import org.mockito.Mockito;
import org.slf4j.Logger;
@@ -102,6 +96,7 @@ 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";
@DataProvider(name = "batch")
public Object[] batchProvider() {
@@ -169,7 +164,9 @@ public void resetDefaultNamespace() throws Exception {
defaultConf.getKeySharedLookAheadMsgInReplayThresholdPerConsumer());
}
- private static final Random random = new Random(System.nanoTime());
+ // Use a fixed seed to make the tests using random values deterministic
+ // When a test fails, it's possible to re-run it to reproduce the issue
+ private static final Random random = new Random(1);
private static final int NUMBER_OF_KEYS = 300;
@Test(dataProvider = "data")
@@ -260,6 +257,8 @@ public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(bo
Consumer consumer3 = createConsumer(topic, KeySharedPolicy.stickyHashRange()
.ranges(Range.of(40001, KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE-1)));
+ StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME);
+
@Cleanup
Producer producer = createProducer(topic, enableBatch);
@@ -269,11 +268,10 @@ public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(bo
for (int i = 0; i < 10; i++) {
for (String key : keys) {
- int slot = Murmur3_32Hash.getInstance().makeHash(key.getBytes())
- % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE;
- if (slot <= 20000) {
+ int stickyKeyHash = selector.makeStickyKeyHash(key.getBytes());
+ if (stickyKeyHash <= 20000) {
consumer1ExpectMessages++;
- } else if (slot <= 40000) {
+ } else if (stickyKeyHash <= 40000) {
consumer2ExpectMessages++;
} else {
consumer3ExpectMessages++;
@@ -383,6 +381,8 @@ public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelect
Consumer consumer3 = createConsumer(topic, KeySharedPolicy.stickyHashRange()
.ranges(Range.of(40001, KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE-1)));
+ StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME);
+
@Cleanup
Producer producer = createProducer(topic, enableBatch);
@@ -396,11 +396,10 @@ public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelect
.send();
String fallbackKey = producer.getProducerName() + "-" + producer.getLastSequenceId();
- int slot = Murmur3_32Hash.getInstance().makeHash(fallbackKey.getBytes())
- % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE;
- if (slot <= 20000) {
+ int stickyKeyHash = selector.makeStickyKeyHash(fallbackKey.getBytes());
+ if (stickyKeyHash <= 20000) {
consumer1ExpectMessages++;
- } else if (slot <= 40000) {
+ } else if (stickyKeyHash <= 40000) {
consumer2ExpectMessages++;
} else {
consumer3ExpectMessages++;
@@ -460,6 +459,8 @@ public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(boole
Consumer