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 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 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); @@ -469,11 +470,10 @@ public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(boole 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++; @@ -502,7 +502,7 @@ public void testDisableKeySharedSubscription() throws PulsarClientException { @Cleanup Consumer c = pulsarClient.newConsumer() .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .ackTimeout(10, TimeUnit.SECONDS) .subscribe(); @@ -540,7 +540,6 @@ public void testCannotUseAcknowledgeCumulative() throws PulsarClientException { @Test(dataProvider = "batch") public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exception { String topic = "testMakingProgressWithSlowerConsumer-" + UUID.randomUUID(); - String slowKey = "slowKey"; List clients = new ArrayList<>(); @@ -556,16 +555,15 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc Consumer c = client.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) - .receiverQueueSize(1) + .receiverQueueSize(100) .messageListener((consumer, msg) -> { try { if (slowKey.equals(msg.getKey())) { // Block the thread to simulate a slow consumer Thread.sleep(10000); } - receivedMessages.incrementAndGet(); consumer.acknowledge(msg); } catch (Exception e) { @@ -576,6 +574,11 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc consumers.add(c); } + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + + org.apache.pulsar.broker.service.Consumer slowConsumer = + selector.select(selector.makeStickyKeyHash(slowKey.getBytes())); + @Cleanup Producer producer = createProducer(topic, enableBatch); @@ -587,18 +590,24 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc int N = 1000; + int nonSlowMessages = 0; + // Then send all the other keys for (int i = 0; i < N; i++) { + String key = String.valueOf(random.nextInt(NUMBER_OF_KEYS)); + if (selector.select(selector.makeStickyKeyHash(key.getBytes())) != slowConsumer) { + // count messages that are not going to the slow consumer + nonSlowMessages++; + } producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(key) .value(i) .send(); } - // Since only 1 out of 10 consumers is stuck, we should be able to receive ~90% messages, - // plus or minus for some skew in the key distribution. + int finalNonSlowMessages = nonSlowMessages; Awaitility.await().untilAsserted(() -> { - assertEquals((double) receivedMessages.get(), N * 0.9, N * 0.3); + assertThat(receivedMessages.get()).isGreaterThanOrEqualTo(finalNonSlowMessages); }); for (Consumer c : consumers) { @@ -614,6 +623,7 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc @Test public void testOrderingWhenAddingConsumers() throws Exception { String topic = "testOrderingWhenAddingConsumers-" + UUID.randomUUID(); + int numberOfKeys = 10; @Cleanup Producer producer = createProducer(topic, false); @@ -623,12 +633,14 @@ public void testOrderingWhenAddingConsumers() throws Exception { for (int i = 0; i < 10; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } - // All the already published messages will be pre-fetched by C1. + PendingAcksMap c1PendingAcks = getDispatcher(topic, SUBSCRIPTION_NAME).getConsumers().get(0).getPendingAcks(); + // Wait until all the already published messages have been pre-fetched by C1. + Awaitility.await().ignoreExceptions().until(() -> c1PendingAcks.size() == 10); // Adding a new consumer. @Cleanup @@ -636,11 +648,14 @@ public void testOrderingWhenAddingConsumers() throws Exception { for (int i = 10; i < 20; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } + Message message = c2.receive(100, TimeUnit.MILLISECONDS); + assertThat(message).describedAs("All keys should be blocked by ").isNull(); + // Closing c1, would trigger all messages to go to c2 c1.close(); @@ -652,6 +667,12 @@ public void testOrderingWhenAddingConsumers() throws Exception { } } + @SneakyThrows + private PersistentStickyKeyDispatcherMultipleConsumers getDispatcher(String topic, String subscription) { + return (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get() + .get().getSubscription(subscription).getDispatcher(); + } + @Test public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { String topic = "testReadAheadWithConfiguredLookAheadLimit-" + UUID.randomUUID(); @@ -665,7 +686,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { @Cleanup Consumer c1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .subscribe(); @@ -683,7 +704,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { @Cleanup Consumer c2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .subscribe(); @@ -701,7 +722,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { Thread.sleep(1000); Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); - PersistentSubscription sub = (PersistentSubscription) t.getSubscription("key_shared"); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(SUBSCRIPTION_NAME); // We need to ensure that dispatcher does not keep to look ahead in the topic, Position readPosition = sub.getCursor().getReadPosition(); @@ -712,6 +733,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { @Test public void testRemoveFirstConsumer() throws Exception { String topic = "testReadAheadWhenAddingConsumers-" + UUID.randomUUID(); + int numberOfKeys = 10; @Cleanup Producer producer = createProducer(topic, false); @@ -719,7 +741,7 @@ public void testRemoveFirstConsumer() throws Exception { @Cleanup Consumer c1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .consumerName("c1") @@ -727,7 +749,7 @@ public void testRemoveFirstConsumer() throws Exception { for (int i = 0; i < 10; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } @@ -740,7 +762,7 @@ public void testRemoveFirstConsumer() throws Exception { @Cleanup Consumer c2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .consumerName("c2") @@ -748,13 +770,13 @@ public void testRemoveFirstConsumer() throws Exception { for (int i = 10; i < 20; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } // C2 will not be able to receive any messages until C1 is done processing whatever he got prefetched - assertNull(c2.receive(100, TimeUnit.MILLISECONDS)); + assertNull(c2.receive(1, TimeUnit.SECONDS)); c1.close(); @@ -777,8 +799,7 @@ public void testHashRangeConflict() throws PulsarClientException { Consumer consumer2 = createFixedHashRangesConsumer(topic, sub, Range.of(100,399)); Assert.assertTrue(consumer2.isConnected()); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) pulsar - .getBrokerService().getTopicReference(topic).get().getSubscription(sub).getDispatcher(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, sub); Assert.assertEquals(dispatcher.getConsumers().size(), 2); try { @@ -887,6 +908,7 @@ public void testAttachKeyToMessageMetadata() throws PulsarClientException { public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { int defaultTTLSec = 3; int totalMessages = 1000; + int numberOfKeys = 50; this.conf.setTtlDurationDefaultInSeconds(defaultTTLSec); final String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); final String subName = "my-sub"; @@ -894,11 +916,15 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { @Cleanup Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) + .consumerName("consumer1") .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subName); + StickyKeyConsumerSelector selector = dispatcher.getSelector(); + @Cleanup Producer producer = pulsarClient.newProducer(Schema.INT32) .topic(topic) @@ -906,20 +932,26 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { for (int i = 0; i < totalMessages; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } - // don't ack the first message - consumer1.receive(); - consumer1.acknowledge(consumer1.receive()); + Set blockedHashes = new HashSet<>(); + // pull up to numberOfKeys messages and don't ack them + for (int i = 0; i < numberOfKeys + 1; i++) { + Message received = consumer1.receive(); + int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); + log.info("Received message {} with sticky key hash: {}", received.getMessageId(), stickyKeyHash); + blockedHashes.add(stickyKeyHash); + } - // The consumer1 and consumer2 should be stuck because of the mark delete position did not move forward. + // The consumer1 and consumer2 should be stuck since all hashes are blocked @Cleanup Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) + .consumerName("consumer2") .subscriptionName(subName) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); @@ -929,11 +961,19 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { received = consumer2.receive(1, TimeUnit.SECONDS); } catch (PulsarClientException ignore) { } - Assert.assertNull(received); + if (received != null) { + int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); + DrainingHashesTracker.DrainingHashEntry entry = + dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + + "included in blockedHashes=%s", + received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); + } @Cleanup Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) + .consumerName("consumer3") .subscriptionName(subName) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); @@ -942,7 +982,14 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { received = consumer3.receive(1, TimeUnit.SECONDS); } catch (PulsarClientException ignore) { } - Assert.assertNull(received); + if (received != null) { + int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); + DrainingHashesTracker.DrainingHashEntry entry = + dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + + "included in blockedHashes=%s", + received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); + } Optional topicRef = pulsar.getBrokerService().getTopic(topic, false).get(); assertTrue(topicRef.isPresent()); @@ -952,14 +999,23 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { // The mark delete position is move forward, so the consumers should receive new messages now. for (int i = 0; i < totalMessages; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } - // Wait broker dispatch messages. - Assert.assertNotNull(consumer2.receive(1, TimeUnit.SECONDS)); - Assert.assertNotNull(consumer3.receive(1, TimeUnit.SECONDS)); + Map receivedMessagesCountByConsumer = new ConcurrentHashMap<>(); + receiveMessages((consumer, message) -> { + consumer.acknowledgeAsync(message); + receivedMessagesCountByConsumer.computeIfAbsent(consumer.getConsumerName(), id -> new AtomicInteger(0)) + .incrementAndGet(); + return true; + }, Duration.ofSeconds(2), consumer1, consumer2, consumer3); + + assertThat(receivedMessagesCountByConsumer.values().stream().mapToInt(AtomicInteger::intValue) + .sum()).isGreaterThanOrEqualTo(totalMessages); + assertThat(receivedMessagesCountByConsumer.values()).allSatisfy( + count -> assertThat(count.get()).isGreaterThan(0)); } @Test(dataProvider = "partitioned") @@ -1151,15 +1207,8 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr producer.send("message".getBytes()); Awaitility.await().untilAsserted(() -> assertNotNull(consumer1.receive(100, TimeUnit.MILLISECONDS))); - CompletableFuture> future = pulsar.getBrokerService().getTopicIfExists(topicName); - assertTrue(future.isDone()); - assertTrue(future.get().isPresent()); - Topic topic = future.get().get(); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subName).getDispatcher(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topicName, subName); assertTrue(dispatcher.isAllowOutOfOrderDelivery()); - assertNull(dispatcher.getLastSentPositionField()); - assertNull(dispatcher.getIndividuallySentPositionsField()); consumer1.close(); final Consumer consumer2 = pulsarClient.newConsumer() @@ -1171,14 +1220,8 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr producer.send("message".getBytes()); Awaitility.await().untilAsserted(() -> assertNotNull(consumer2.receive(100, TimeUnit.MILLISECONDS))); - future = pulsar.getBrokerService().getTopicIfExists(topicName); - assertTrue(future.isDone()); - assertTrue(future.get().isPresent()); - topic = future.get().get(); - dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subName).getDispatcher(); + dispatcher = getDispatcher(topicName, subName); assertFalse(dispatcher.isAllowOutOfOrderDelivery()); - assertNotNull(dispatcher.getLastSentPositionField()); - assertNotNull(dispatcher.getIndividuallySentPositionsField()); consumer2.close(); } @@ -1250,7 +1293,7 @@ public void testCheckConsumersWithSameName() throws Exception { } }); - l.await(); + l.await(10, TimeUnit.SECONDS); } @DataProvider(name = "preSend") @@ -1258,366 +1301,6 @@ private Object[][] preSendProvider() { return new Object[][] { { false }, { true } }; } - @Test(timeOut = 30_000, dataProvider = "preSend") - public void testCheckBetweenSkippingAndRecentlyJoinedConsumers(boolean preSend) throws Exception { - conf.setSubscriptionKeySharedUseConsistentHashing(true); - - final String topicName = "persistent://public/default/recently-joined-consumers-" + UUID.randomUUID(); - final String subName = "my-sub"; - - @Cleanup - final Producer p = pulsarClient.newProducer(Schema.STRING) - .topic(topicName) - .create(); - if (preSend) { - // verify that the test succeeds even if the topic has a message - p.send("msg"); - } - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.STRING) - .topic(topicName) - .subscriptionInitialPosition(SubscriptionInitialPosition.Latest) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)); - - // create 2 consumers - final String c1ConsumerName = "c1"; - @Cleanup - final Consumer c1 = cb.get().consumerName(c1ConsumerName).receiverQueueSize(1).subscribe(); - @Cleanup - final Consumer c2 = cb.get().consumerName("c2").receiverQueueSize(1000).subscribe(); - - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - final Field recentlyJoinedConsumersField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("recentlyJoinedConsumers"); - recentlyJoinedConsumersField.setAccessible(true); - final LinkedHashMap recentlyJoinedConsumers = (LinkedHashMap) recentlyJoinedConsumersField.get(dispatcher); - final String keyA = "key-a"; - final int hashA = Murmur3_32Hash.getInstance().makeHash(keyA.getBytes()); - final Map hashConsumerMap = new HashMap<>(); - hashConsumerMap.put(hashA, c1.getConsumerName()); - - // enforce the selector will return c1 if keyA - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - // send and receive - Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getConsumers().stream().filter(c -> c.getConsumerName().equals(c1ConsumerName)).findFirst().get().getAvailablePermits(), 1)); - final MessageIdImpl msg0Id = (MessageIdImpl) p.newMessage().key(keyA).value("msg-0").send(); - Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getConsumers().stream().filter(c -> c.getConsumerName().equals(c1ConsumerName)).findFirst().get().getAvailablePermits(), 0)); - - final MessageIdImpl msg1Id = (MessageIdImpl) p.newMessage().key(keyA).value("msg-1").send(); - Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getMsgBacklog(), 2)); - - final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class - .getDeclaredField("redeliveryMessages"); - redeliveryMessagesField.setAccessible(true); - final MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField.get(dispatcher); - - final Set replayMsgSet = redeliveryMessages.getMessagesToReplayNow(3, item -> true); - assertEquals(replayMsgSet.size(), 1); - final Position replayMsg = replayMsgSet.stream().findAny().get(); - assertEquals(replayMsg, PositionFactory.create(msg1Id.getLedgerId(), msg1Id.getEntryId())); - - // add c3 - final String c3ConsumerName = "c3"; - hashConsumerMap.put(hashA, c3ConsumerName); - @Cleanup - final Consumer c3 = cb.get().consumerName(c3ConsumerName).subscribe(); - final List> c3Msgs = new ArrayList<>(); - final org.apache.pulsar.broker.service.Consumer c3Broker = dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(c3ConsumerName)).findFirst().get(); - assertEquals(recentlyJoinedConsumers.get(c3Broker), PositionFactory.create(msg0Id.getLedgerId(), msg0Id.getEntryId())); - - // None of messages are sent to c3. - Message c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); - assertNull(c3Msg); - - // Disconnect c1 - c1.close(); - - c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); - assertNotNull(c3Msg); - c3Msgs.add(c3Msg); - // The mark delete position will move forward. Then remove c3 from recentlyJoinedConsumers. - c3.acknowledge(c3Msg); - Awaitility.await().untilAsserted(() -> assertNull(recentlyJoinedConsumers.get(c3Broker))); - c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); - assertNotNull(c3Msg); - c3Msgs.add(c3Msg); - c3.acknowledge(c3Msg); - - // check ordering - assertTrue(c3Msgs.get(0).getMessageId().compareTo(c3Msgs.get(1).getMessageId()) < 0); - } - - @Test(timeOut = 30_000) - public void testLastSentPositionWhenRecreatingDispatcher() throws Exception { - // The lastSentPosition and individuallySentPositions should be initialized - // by the markDeletedPosition and individuallyDeletedMessages. - final String topicName = "persistent://public/default/rewind-" + UUID.randomUUID(); - final String subName = "my-sub"; - - final int numMessages = 9; - final List keys = Arrays.asList("key-a", "key-b", "key-c"); - final AtomicInteger receiveCounter = new AtomicInteger(); - final AtomicInteger ackCounter = new AtomicInteger(); - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.INT32) - .topic(topicName) - .enableBatching(false) - .create(); - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)); - - @Cleanup - final Consumer c1 = cb.get().messageListener((c, msg) -> { - if (keys.get(0).equals(msg.getKey())) { - try { - c.acknowledge(msg); - ackCounter.getAndIncrement(); - } catch (PulsarClientException e) { - fail(e.getMessage()); - } - } - receiveCounter.getAndIncrement(); - }).subscribe(); - - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - final ManagedCursorImpl cursor = (ManagedCursorImpl) ((PersistentSubscription) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName)).getCursor(); - final ManagedLedgerImpl ledger = (ManagedLedgerImpl) cursor.getManagedLedger(); - - MessageIdImpl msgId = null; - for (int i = 0; i < numMessages; i++) { - msgId = (MessageIdImpl) producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); - } - - // wait for consumption - Awaitility.await().untilAsserted(() -> assertEquals(receiveCounter.get(), numMessages)); - assertEquals(ackCounter.get(), numMessages / keys.size()); - assertEquals(dispatcher.getLastSentPositionField(), PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); - assertTrue(individuallySentPositionsField.isEmpty()); - receiveCounter.set(0); - ackCounter.set(0); - - // create expected values - final Position expectedLastSentPosition = ledger.getNextValidPosition(cursor.getMarkDeletedPosition()); - final ConcurrentOpenLongPairRangeSet - expectedIndividuallySentPositions = new ConcurrentOpenLongPairRangeSet<>(4096, PositionFactory::create); - cursor.getIndividuallyDeletedMessagesSet().forEach(range -> { - final Position lower = range.lowerEndpoint(); - final Position upper = range.upperEndpoint(); - expectedIndividuallySentPositions.addOpenClosed(lower.getLedgerId(), lower.getEntryId(), upper.getLedgerId(), upper.getEntryId()); - return true; - }); - - // modify subscription type to close current dispatcher - admin.topics().createSubscription(topicName, "sub-alt", MessageId.earliest); - c1.close(); - @Cleanup - final Consumer c2 = pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Exclusive) - .subscribe(); - c2.close(); - assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getType(), SubscriptionType.Exclusive.toString()); - - @Cleanup - final Consumer c3 = cb.get().receiverQueueSize(0).subscribe(); - dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - - assertNull(dispatcher.getLastSentPositionField()); - assertTrue(individuallySentPositionsField.isEmpty()); - - assertNotNull(c3.receive()); - - // validate the individuallySentPosition is initialized by the individuallyDeletedMessages - // if it is not initialized expectedly, it has sent-hole of key-c messages because key-c messages are not scheduled to be dispatched to some consumer(already acked). - assertEquals(dispatcher.getLastSentPositionField(), expectedLastSentPosition); - assertEquals(individuallySentPositionsField.toString(), expectedIndividuallySentPositions.toString()); - } - - @Test(timeOut = 30_000) - public void testLastSentPositionWhenResettingCursor() throws Exception { - // The lastSentPosition and individuallySentPositions should be cleared if reset-cursor operation is executed. - final String nsName = "public/default"; - final String topicName = "persistent://" + nsName + "/reset-cursor-" + UUID.randomUUID(); - final String subName = "my-sub"; - - final int numMessages = 10; - final List keys = Arrays.asList("key-a", "key-b"); - final AtomicInteger ackCounter = new AtomicInteger(); - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.INT32) - .topic(topicName) - .enableBatching(false) - .create(); - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .receiverQueueSize(0) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)); - - @Cleanup - final Consumer c1 = cb.get().consumerName("c1").subscribe(); - @Cleanup - final Consumer c2 = cb.get().consumerName("c2").subscribe(); - - // set retention policy - admin.namespaces().setRetention(nsName, new RetentionPolicies(1, 1024 * 1024)); - - // enforce the selector will return c1 if keys.get(0) - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - final int hashA = Murmur3_32Hash.getInstance().makeHash(keys.get(0).getBytes()); - final Map hashConsumerMap = new HashMap<>(); - hashConsumerMap.put(hashA, c1.getConsumerName()); - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - for (int i = 0; i < numMessages; i++) { - producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); - } - - // consume some messages - for (int i = 0; i < numMessages / keys.size(); i++) { - final Message msg = c2.receive(); - if (msg != null) { - c2.acknowledge(msg); - ackCounter.getAndIncrement(); - } - } - assertEquals(ackCounter.get(), numMessages / keys.size()); - - // store current lastSentPosition for comparison - final LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - assertNotNull(dispatcher.getLastSentPositionField()); - assertFalse(individuallySentPositionsField.isEmpty()); - - // reset cursor and receive a message - admin.topics().resetCursor(topicName, subName, MessageId.earliest, true); - - // validate the lastSentPosition and individuallySentPositions are cleared after resetting cursor - assertNull(dispatcher.getLastSentPositionField()); - assertTrue(individuallySentPositionsField.isEmpty()); - } - - @Test(timeOut = 30_000) - public void testLastSentPositionWhenSkipping() throws Exception { - // The lastSentPosition and individuallySentPositions should be updated if skip operation is executed. - // There are updated to follow the new markDeletedPosition. - final String topicName = "persistent://public/default/skip-" + UUID.randomUUID(); - final String subName = "my-sub"; - - final int numMessages = 10; - final List keys = Arrays.asList("key-a", "key-b"); - final int numSkip = 2; - final AtomicInteger ackCounter = new AtomicInteger(); - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.INT32) - .topic(topicName) - .enableBatching(false) - .create(); - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)) - .receiverQueueSize(0); - - @Cleanup - final Consumer c1 = cb.get().consumerName("c1").subscribe(); - @Cleanup - final Consumer c2 = cb.get().consumerName("c2").subscribe(); - - // enforce the selector will return c1 if keys.get(0) - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - final int hashA = Murmur3_32Hash.getInstance().makeHash(keys.get(0).getBytes()); - final Map hashConsumerMap = new HashMap<>(); - hashConsumerMap.put(hashA, c1.getConsumerName()); - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - final List positionList = new ArrayList<>(); - for (int i = 0; i < numMessages; i++) { - final MessageIdImpl msgId = (MessageIdImpl) producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); - positionList.add(PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); - } - - // consume some messages - for (int i = 0; i < numSkip; i++) { - final Message msg = c2.receive(); - if (msg != null) { - c2.acknowledge(msg); - ackCounter.getAndIncrement(); - } - } - assertEquals(ackCounter.get(), numSkip); - final ManagedCursorImpl managedCursor = ((ManagedCursorImpl) ((PersistentSubscription) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName)).getCursor()); - Awaitility.await().untilAsserted(() -> assertEquals(managedCursor.getIndividuallyDeletedMessagesSet().size(), 2)); - - // store current lastSentPosition for comparison - final Position lastSentPositionBeforeSkip = dispatcher.getLastSentPositionField(); - final LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - assertNotNull(lastSentPositionBeforeSkip); - assertFalse(individuallySentPositionsField.isEmpty()); - - // skip messages and receive a message - admin.topics().skipMessages(topicName, subName, numSkip); - final MessageIdImpl msgIdAfterSkip = (MessageIdImpl) c1.receive().getMessageId(); - final Position positionAfterSkip = PositionFactory.create(msgIdAfterSkip.getLedgerId(), - msgIdAfterSkip.getEntryId()); - assertEquals(positionAfterSkip, positionList.get(4)); - - // validate the lastSentPosition is updated to the new markDeletedPosition - // validate the individuallySentPositions is updated expectedly (removeAtMost the new markDeletedPosition) - final Position lastSentPosition = dispatcher.getLastSentPositionField(); - assertNotNull(lastSentPosition); - assertTrue(lastSentPosition.compareTo(lastSentPositionBeforeSkip) > 0); - assertEquals(lastSentPosition, positionList.get(4)); - assertTrue(individuallySentPositionsField.isEmpty()); - } - private KeySharedMode getKeySharedModeOfSubscription(Topic topic, String subscription) { if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.persistent)) { return ((PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subscription) @@ -1665,7 +1348,7 @@ private Consumer createConsumer(String topic, KeySharedPolicy keyShared throws PulsarClientException { ConsumerBuilder builder = pulsarClient.newConsumer(Schema.INT32); builder.topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .ackTimeout(3, TimeUnit.SECONDS); if (keySharedPolicy != null) { @@ -1927,8 +1610,8 @@ public void testStickyKeyRangesRestartConsumers() throws Exception { }}); // wait for some messages to be received by both of the consumers - count1.await(); - count2.await(); + count1.await(5, TimeUnit.SECONDS); + count2.await(5, TimeUnit.SECONDS); consumer1.close(); consumer2.close(); @@ -1974,7 +1657,7 @@ public void testStickyKeyRangesRestartConsumers() throws Exception { }) .subscribe(); // wait for all the messages to be delivered - count3.await(); + count3.await(20, TimeUnit.SECONDS); assertTrue(sentMessages.isEmpty(), "didn't receive " + sentMessages); producerFuture.get(); @@ -2085,7 +1768,8 @@ private AtomicInteger injectReplayReadCounter(String topicName, String cursorNam @Test public void testNoRepeatedReadAndDiscard() throws Exception { int delayedMessages = 100; - final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + int numberOfKeys = delayedMessages; + final String topic = newUniqueName("persistent://public/default/tp"); final String subName = "my-sub"; admin.topics().createNonPartitionedTopic(topic); AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); @@ -2095,7 +1779,7 @@ public void testNoRepeatedReadAndDiscard() throws Exception { Producer producer = pulsarClient.newProducer(Schema.INT32).topic(topic).enableBatching(false).create(); for (int i = 0; i < delayedMessages; i++) { MessageId messageId = producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(random.nextInt(numberOfKeys))) .value(100 + i) .send(); log.info("Published message :{}", messageId); @@ -2103,12 +1787,14 @@ public void testNoRepeatedReadAndDiscard() throws Exception { producer.close(); // Make ack holes. + @Cleanup Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); + @Cleanup Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) @@ -2136,7 +1822,7 @@ public void testNoRepeatedReadAndDiscard() throws Exception { redeliverConsumer = consumer1; } - // consumer3 will be added to the "recentJoinedConsumers". + @Cleanup Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) @@ -2145,17 +1831,10 @@ public void testNoRepeatedReadAndDiscard() throws Exception { .subscribe(); redeliverConsumer.close(); + Thread.sleep(5000); // Verify: no repeated Read-and-discard. - Thread.sleep(5 * 1000); int maxReplayCount = delayedMessages * 2; - log.info("Reply read count: {}", replyReadCounter.get()); - assertTrue(replyReadCounter.get() < maxReplayCount); - - // cleanup. - consumer1.close(); - consumer2.close(); - consumer3.close(); - admin.topics().delete(topic, false); + assertThat(replyReadCounter.get()).isLessThanOrEqualTo(maxReplayCount); } @DataProvider(name = "allowKeySharedOutOfOrder") @@ -2184,7 +1863,7 @@ public Object[][] allowKeySharedOutOfOrder() { public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedOutOfOrder) throws Exception { final int messagesSentPerTime = 100; final Set totalReceivedMessages = new TreeSet<>(); - final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topic = newUniqueName("persistent://public/default/tp"); final String subName = "my-sub"; admin.topics().createNonPartitionedTopic(topic); AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); @@ -2243,7 +1922,7 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO msgList2.add(msg2); } Message msg3 = consumer3.receive(1, TimeUnit.SECONDS); - if (msg2 != null) { + if (msg3 != null) { totalReceivedMessages.add(msg3.getValue()); msgList3.add(msg3); } @@ -2251,23 +1930,35 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO Consumer consumerWillBeClose = null; Consumer consumerAlwaysAck = null; Consumer consumerStuck = null; + Runnable consumerStuckAckHandler; + if (!msgList1.isEmpty()) { msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); consumerAlwaysAck = consumer1; consumerWillBeClose = consumer2; consumerStuck = consumer3; + consumerStuckAckHandler = () -> { + msgList3.forEach(msg -> consumer3.acknowledgeAsync(msg)); + }; } else if (!msgList2.isEmpty()){ msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); consumerAlwaysAck = consumer2; consumerWillBeClose = consumer3; consumerStuck = consumer1; + consumerStuckAckHandler = () -> { + msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); + }; } else { msgList3.forEach(msg -> consumer3.acknowledgeAsync(msg)); consumerAlwaysAck = consumer3; consumerWillBeClose = consumer1; consumerStuck = consumer2; + consumerStuckAckHandler = () -> { + msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); + }; } + // 2. Add consumer4 after "consumerWillBeClose" was close, and consumer4 will be stuck due to the mechanism // "recentlyJoinedConsumers". Consumer consumer4 = pulsarClient.newConsumer(Schema.INT32) @@ -2314,6 +2005,7 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO log.info("Reply read count: {}", replyReadCounter.get()); assertTrue(replyReadCounter.get() < maxReplayCount); // Verify: at last, all messages will be received. + consumerStuckAckHandler.run(); ReceivedMessages receivedMessages = ackAllMessages(consumerAlwaysAck, consumerStuck, consumer4); totalReceivedMessages.addAll(receivedMessages.messagesReceived.stream().map(p -> p.getRight()).collect( Collectors.toList())); @@ -2340,7 +2032,7 @@ public void testReadAheadLimit() throws Exception { Producer producer = createProducer(topic, false); // create a consumer and close it to create a subscription - String subscriptionName = "key_shared"; + String subscriptionName = SUBSCRIPTION_NAME; pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subscriptionName) @@ -2348,11 +2040,7 @@ public void testReadAheadLimit() throws Exception { .subscribe() .close(); - Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); - PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscriptionName); - // get the dispatcher reference - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subscriptionName); // create a function to use for checking the number of messages in replay Runnable checkLimit = () -> { @@ -2394,8 +2082,7 @@ public void testReadAheadLimit() throws Exception { for (int i = 0; i < numberOfKeys; i++) { String key = String.valueOf(i); byte[] keyBytes = key.getBytes(UTF_8); - int hash = StickyKeyConsumerSelector.makeStickyKeyHash(keyBytes); - if (dispatcher.getSelector().select(hash).consumerName().equals("c2")) { + if (dispatcher.getSelector().select(keyBytes).consumerName().equals("c2")) { keysForC2.add(key); } } @@ -2453,4 +2140,171 @@ public void testReadAheadLimit() throws Exception { }, Duration.ofSeconds(2), c1, c2, c3); assertEquals(remainingMessageValues, Collections.emptySet()); } + + @SneakyThrows + private StickyKeyConsumerSelector getSelector(String topic, String subscription) { + Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + return dispatcher.getSelector(); + } + + // This test case simulates a rolling restart scenario with behaviors that can trigger out-of-order issues. + // In earlier versions of Pulsar, this issue occurred in about 25% of cases. + // To increase the probability of reproducing the issue, use the invocationCount parameter. + @Test//(invocationCount = 50) + public void testOrderingAfterReconnects() throws Exception { + String topic = newUniqueName("testOrderingAfterReconnects"); + int numberOfKeys = 1000; + long pauseTime = 100L; + + @Cleanup + Producer producer = createProducer(topic, false); + + // create a consumer and close it to create a subscription + pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe() + .close(); + + Set remainingMessageValues = new HashSet<>(); + Map> keyPositions = new HashMap<>(); + BiFunction, Message, Boolean> messageHandler = (consumer, msg) -> { + synchronized (this) { + consumer.acknowledgeAsync(msg); + String key = msg.getKey(); + MessageIdAdv msgId = (MessageIdAdv) msg.getMessageId(); + Position currentPosition = PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId()); + Pair prevPair = keyPositions.get(key); + if (prevPair != null && prevPair.getLeft().compareTo(currentPosition) > 0) { + log.error("key: {} value: {} prev: {}/{} current: {}/{}", key, msg.getValue(), prevPair.getLeft(), + prevPair.getRight(), currentPosition, consumer.getConsumerName()); + fail("out of order"); + } + keyPositions.put(key, Pair.of(currentPosition, consumer.getConsumerName())); + boolean removed = remainingMessageValues.remove(msg.getValue()); + if (!removed) { + // duplicates are possible during reconnects, this is not an error + log.warn("Duplicate message: {} value: {}", msg.getMessageId(), msg.getValue()); + } + return true; + } + }; + + // Adding a new consumer. + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .startPaused(true) // start paused + .subscribe(); + + @Cleanup + Consumer c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(500) // use large receiver queue size + .subscribe(); + + @Cleanup + Consumer c3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .startPaused(true) // start paused + .subscribe(); + + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + + // find keys that will be assigned to c2 + List keysForC2 = new ArrayList<>(); + for (int i = 0; i < numberOfKeys; i++) { + String key = String.valueOf(i); + byte[] keyBytes = key.getBytes(UTF_8); + int hash = selector.makeStickyKeyHash(keyBytes); + if (selector.select(hash).consumerName().equals("c2")) { + keysForC2.add(key); + } + } + + // produce messages with keys that all get assigned to c2 + for (int i = 0; i < 1000; i++) { + String key = keysForC2.get(random.nextInt(keysForC2.size())); + //log.info("Producing message with key: {} value: {}", key, i); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + } + + Thread.sleep(2 * pauseTime); + // close c2 + c2.close(); + Thread.sleep(pauseTime); + // resume c1 and c3 + c1.resume(); + c3.resume(); + Thread.sleep(pauseTime); + // reconnect c2 + c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .subscribe(); + // close and reconnect c1 + c1.close(); + Thread.sleep(pauseTime); + c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .subscribe(); + // close and reconnect c3 + c3.close(); + Thread.sleep(pauseTime); + c3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .subscribe(); + + logTopicStats(topic); + + // produce more messages + for (int i = 1000; i < 2000; i++) { + String key = String.valueOf(random.nextInt(numberOfKeys)); + //log.info("Producing message with key: {} value: {}", key, i); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + } + + // consume the messages + receiveMessages(messageHandler, Duration.ofSeconds(2), c1, c2, c3); + + try { + assertEquals(remainingMessageValues, Collections.emptySet()); + } finally { + logTopicStats(topic); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java index 7889b19e5b29e..704af89777f05 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java @@ -18,18 +18,25 @@ */ package org.apache.pulsar.client.impl; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; +import static org.assertj.core.api.Assertions.assertThat; import com.google.common.collect.Sets; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.RandomStringUtils; +import lombok.SneakyThrows; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -40,7 +47,6 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.SubscriptionType; -import org.apache.pulsar.common.util.Murmur3_32Hash; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -65,33 +71,76 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + enum KeySharedSelectorType { + AutoSplit_ConsistentHashing(true), AutoSplit_Classic(true), Sticky(false); + final boolean autoSplit; + + KeySharedSelectorType(boolean autoSplit) { + this.autoSplit = autoSplit; + } + } + @DataProvider public Object[][] subType() { - return new Object[][] { { SubscriptionType.Shared }, { SubscriptionType.Key_Shared } }; + return new Object[][] { + { SubscriptionType.Shared, null }, + { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_ConsistentHashing }, + { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_Classic }, + { SubscriptionType.Key_Shared, KeySharedSelectorType.Sticky } + }; } - @Test(dataProvider = "subType") - public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType) + @Test(dataProvider = "subType", timeOut = 30000) + public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType, + KeySharedSelectorType selectorType) throws PulsarClientException { + if (selectorType == KeySharedSelectorType.AutoSplit_Classic) { + conf.setSubscriptionKeySharedUseConsistentHashing(false); + } + final int totalMsg = 1000; - String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5); - Map, List> nameToId = new ConcurrentHashMap<>(); + String topic = newUniqueName("broker-close-test"); + String subscriptionName = "sub-1"; + Map, List> unackedMessages = new ConcurrentHashMap<>(); Set pubMessages = Sets.newConcurrentHashSet(); Set recMessages = Sets.newConcurrentHashSet(); AtomicLong lastActiveTime = new AtomicLong(); AtomicBoolean canAcknowledgement = new AtomicBoolean(false); + if (subscriptionType == SubscriptionType.Key_Shared) { + // create and close consumer to create the dispatcher so that the selector can be used + ConsumerBuilder consumerBuilder = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName(subscriptionName) + .subscriptionType(subscriptionType); + if (subscriptionType == SubscriptionType.Key_Shared) { + if (selectorType.autoSplit) { + consumerBuilder.keySharedPolicy(KeySharedPolicy.autoSplitHashRange()); + } else { + consumerBuilder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(Range.of(0, 65535))); + } + } + consumerBuilder + .subscribe() + .close(); + } + List> consumerList = new ArrayList<>(); - // create 3 consumers - for (int i = 0; i < 3; i++) { + int consumerCount = 3; + + Range[] ranges = null; + if (subscriptionType == SubscriptionType.Key_Shared && !selectorType.autoSplit) { + ranges = splitRange(getSelector(topic, subscriptionName).getKeyHashRange(), consumerCount); + } + + for (int i = 0; i < consumerCount; i++) { ConsumerBuilder builder = pulsarClient.newConsumer() .topic(topic) - .subscriptionName("sub-1") + .consumerName("consumer-" + i) + .subscriptionName(subscriptionName) .subscriptionType(subscriptionType) .messageListener((consumer, msg) -> { lastActiveTime.set(System.currentTimeMillis()); - nameToId.computeIfAbsent(consumer, (k) -> new ArrayList<>()) - .add(msg.getMessageId()); recMessages.add(msg.getMessageId()); if (canAcknowledgement.get()) { try { @@ -99,19 +148,31 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc } catch (PulsarClientException e) { throw new RuntimeException(e); } + } else { + unackedMessages.computeIfAbsent(consumer, + (k) -> Collections.synchronizedList(new ArrayList<>())) + .add(msg.getMessageId()); } }); if (subscriptionType == SubscriptionType.Key_Shared) { - // ensure every consumer can be distributed messages - int hash = Murmur3_32Hash.getInstance().makeHash(("key-" + i).getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; - builder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(Range.of(hash, hash))); + if (selectorType.autoSplit) { + builder.keySharedPolicy(KeySharedPolicy.autoSplitHashRange()); + } else { + builder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(ranges[i])); + } } consumerList.add(builder.subscribe()); } + String[] keys = new String[consumerCount]; + for (int i = 0; i < consumerCount; i++) { + keys[i] = subscriptionType == SubscriptionType.Key_Shared ? + generateKeyForConsumer(getSelector(topic, subscriptionName), + consumerList.get(i).getConsumerName()) : "key-" + i; + } + Producer producer = pulsarClient.newProducer() .topic(topic) .enableBatching(true) @@ -122,42 +183,39 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc .create(); for (int i = 0; i < totalMsg; i++) { - byte[] msg = UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8); - producer.newMessage().key("key-" + (i % 3)).value(msg) + producer.newMessage() + .key(keys[i % consumerCount]) + .value(("message-" + i).getBytes(StandardCharsets.UTF_8)) .sendAsync().thenAccept(pubMessages::add); } + producer.flush(); + // Wait for all consumers can not read more messages. the consumers are stuck by max unacked messages. - Awaitility.await() - .pollDelay(5, TimeUnit.SECONDS) - .until(() -> - (System.currentTimeMillis() - lastActiveTime.get()) > TimeUnit.SECONDS.toMillis(5)); + waitUntilLastActiveTimeNoLongerGetsUpdated(lastActiveTime); // All consumers can acknowledge messages as they continue to receive messages. canAcknowledgement.set(true); // Acknowledgment of currently received messages to get out of stuck state due to unack message - for (Map.Entry, List> entry : nameToId.entrySet()) { + for (Map.Entry, List> entry : unackedMessages.entrySet()) { Consumer consumer = entry.getKey(); - consumer.acknowledge(entry.getValue()); + List messageIdList = entry.getValue(); + consumer.acknowledge(messageIdList); } + // refresh active time lastActiveTime.set(System.currentTimeMillis()); // Wait for all consumers to continue receiving messages. - Awaitility.await() - .atMost(30, TimeUnit.SECONDS) - .pollDelay(5, TimeUnit.SECONDS) - .until(() -> - (System.currentTimeMillis() - lastActiveTime.get()) > TimeUnit.SECONDS.toMillis(5)); + waitUntilLastActiveTimeNoLongerGetsUpdated(lastActiveTime); logTopicStats(topic); //Determine if all messages have been received. //If the dispatcher is stuck, we can not receive enough messages. Assert.assertEquals(totalMsg, pubMessages.size()); - Assert.assertEquals(recMessages.size(), pubMessages.size()); - Assert.assertTrue(recMessages.containsAll(pubMessages)); + assertThat(recMessages).containsExactlyInAnyOrderElementsOf(pubMessages); // cleanup producer.close(); @@ -165,4 +223,43 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc consumer.close(); } } + + private Range[] splitRange(Range keyHashRange, int consumerCount) { + Range[] ranges = new Range[consumerCount]; + int start = keyHashRange.getStart(); + for (int i = 0; i < consumerCount; i++) { + int end = Math.min(start + keyHashRange.size() / consumerCount, keyHashRange.getEnd()); + ranges[i] = Range.of(start, end); + start = end + 1; + } + return ranges; + } + + private String generateKeyForConsumer(StickyKeyConsumerSelector selector, + String consumerName) { + int i = 0; + while (!Thread.currentThread().isInterrupted()) { + String key = "key" + i++; + org.apache.pulsar.broker.service.Consumer selectedConsumer = selector.select(key.getBytes(UTF_8)); + if (selectedConsumer != null && selectedConsumer.consumerName().equals(consumerName)) { + return key; + } + } + return null; + } + + private static void waitUntilLastActiveTimeNoLongerGetsUpdated(AtomicLong lastActiveTime) { + Awaitility.await() + .pollInterval(100, TimeUnit.MILLISECONDS) + .until(() -> System.currentTimeMillis() - lastActiveTime.get() > TimeUnit.SECONDS.toMillis(1)); + } + + @SneakyThrows + private StickyKeyConsumerSelector getSelector(String topic, String subscription) { + Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + return dispatcher.getSelector(); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index e307e41862e74..7b7c1f5765cc5 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -118,15 +118,6 @@ public interface SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ String getKeySharedMode(); - /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ - Map getConsumersAfterMarkDeletePosition(); - - /** The last sent position of the cursor. This is for Key_Shared subscription. */ - String getLastSentPosition(); - - /** Set of individually sent ranges. This is for Key_Shared subscription. */ - String getIndividuallySentPositions(); - /** SubscriptionProperties (key/value strings) associated with this subscribe. */ Map getSubscriptionProperties(); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java index 488083f484b76..cbca1ef8f06bd 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java @@ -93,4 +93,12 @@ public int compareTo(Range o) { } return result; } + + public boolean contains(int value) { + return value >= start && value <= end; + } + + public int size() { + return end - start + 1; + } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java index 610c782518348..50168221fea37 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java @@ -61,4 +61,44 @@ public void testIntersect() { public void testInvalid() { Range.of(0, -5); } + + @Test + public void testCompareTo() { + Range range1 = Range.of(0, 5); + Range range2 = Range.of(0, 5); + Range range3 = Range.of(0, 10); + Range range4 = Range.of(5, 10); + + Assert.assertEquals(0, range1.compareTo(range2)); + Assert.assertTrue(range1.compareTo(range3) < 0); + Assert.assertTrue(range3.compareTo(range1) > 0); + Assert.assertTrue(range1.compareTo(range4) < 0); + Assert.assertTrue(range4.compareTo(range1) > 0); + } + + @Test + public void testContains() { + Range range = Range.of(0, 5); + + Assert.assertTrue(range.contains(0)); + Assert.assertTrue(range.contains(3)); + Assert.assertTrue(range.contains(5)); + Assert.assertFalse(range.contains(-1)); + Assert.assertFalse(range.contains(6)); + } + + @Test + public void testSize() { + Range range = Range.of(0, 0); + Assert.assertEquals(1, range.size()); + + range = Range.of(0, 1); + Assert.assertEquals(2, range.size()); + + range = Range.of(0, 5); + Assert.assertEquals(6, range.size()); + + range = Range.of(3, 3); + Assert.assertEquals(1, range.size()); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index 977ed28e86814..4206a4aa8d61b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.ArrayList; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -126,15 +125,9 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ public String keySharedMode; - /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ - public Map consumersAfterMarkDeletePosition; - /** The last sent position of the cursor. This is for Key_Shared subscription. */ public String lastSentPosition; - /** Set of individually sent ranges. This is for Key_Shared subscription. */ - public String individuallySentPositions; - /** The number of non-contiguous deleted messages ranges. */ public int nonContiguousDeletedMessagesRanges; @@ -160,7 +153,6 @@ public class SubscriptionStatsImpl implements SubscriptionStats { public SubscriptionStatsImpl() { this.consumers = new ArrayList<>(); - this.consumersAfterMarkDeletePosition = new LinkedHashMap<>(); this.subscriptionProperties = new HashMap<>(); this.bucketDelayedIndexStats = new HashMap<>(); } @@ -185,7 +177,6 @@ public void reset() { lastExpireTimestamp = 0L; lastMarkDeleteAdvancedTimestamp = 0L; consumers.clear(); - consumersAfterMarkDeletePosition.clear(); nonContiguousDeletedMessagesRanges = 0; nonContiguousDeletedMessagesRangesSerializedSize = 0; earliestMsgPublishTimeInBacklog = 0L; @@ -231,7 +222,6 @@ public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { } } this.allowOutOfOrderDelivery |= stats.allowOutOfOrderDelivery; - this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; if (this.earliestMsgPublishTimeInBacklog != 0 && stats.earliestMsgPublishTimeInBacklog != 0) { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 15b5676094ec1..8635368f00f0b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -1970,27 +1970,37 @@ public static MessageMetadata peekAndCopyMessageMetadata( return metadata; } - private static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); + public static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); + public static byte[] peekStickyKey(ByteBuf metadataAndPayload, String topic, String subscription) { try { int readerIdx = metadataAndPayload.readerIndex(); - MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); + MessageMetadata metadata = parseMessageMetadata(metadataAndPayload); metadataAndPayload.readerIndex(readerIdx); - if (metadata.hasOrderingKey()) { - return metadata.getOrderingKey(); - } else if (metadata.hasPartitionKey()) { - if (metadata.isPartitionKeyB64Encoded()) { - return Base64.getDecoder().decode(metadata.getPartitionKey()); - } - 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 resolveStickyKey(metadata); } catch (Throwable t) { log.error("[{}] [{}] Failed to peek sticky key from the message metadata", topic, subscription, t); + return NONE_KEY; + } + } + + public static byte[] resolveStickyKey(MessageMetadata metadata) { + byte[] stickyKey; + if (metadata.hasOrderingKey()) { + stickyKey = metadata.getOrderingKey(); + } else if (metadata.hasPartitionKey()) { + if (metadata.isPartitionKeyB64Encoded()) { + stickyKey = Base64.getDecoder().decode(metadata.getPartitionKey()); + } else { + stickyKey = metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); + } + } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { + String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); + stickyKey = fallbackKey.getBytes(StandardCharsets.UTF_8); + } else { + stickyKey = NONE_KEY; } - return Commands.NONE_KEY; + return stickyKey; } public static int getCurrentProtocolVersion() { diff --git a/pulsar-common/src/main/resources/findbugsExclude.xml b/pulsar-common/src/main/resources/findbugsExclude.xml index df161c4b621a7..b3e511006bce3 100644 --- a/pulsar-common/src/main/resources/findbugsExclude.xml +++ b/pulsar-common/src/main/resources/findbugsExclude.xml @@ -53,4 +53,9 @@ + + + + +