From 329012652dc51afea6ec8fa3c32e9707edf8372c Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Sat, 13 Apr 2019 17:19:00 -0700 Subject: [PATCH 01/15] Delayed message delivery implementation --- conf/broker.conf | 10 + .../pulsar/broker/ServiceConfiguration.java | 7 + .../apache/pulsar/broker/PulsarService.java | 22 ++- .../pulsar/broker/service/Consumer.java | 37 +++- .../pulsar/broker/service/Dispatcher.java | 13 ++ .../pulsar/broker/service/Subscription.java | 2 + .../persistent/DelayedDeliveryTracker.java | 180 +++++++++++++++++ ...PersistentDispatcherMultipleConsumers.java | 108 +++++++--- .../persistent/PersistentSubscription.java | 19 +- .../prometheus/AggregatedNamespaceStats.java | 1 + .../AggregatedSubscriptionStats.java | 2 + .../prometheus/NamespaceStatsAggregator.java | 1 + .../broker/stats/prometheus/TopicStats.java | 1 + .../service/PersistentTopicE2ETest.java | 2 +- .../persistent/DelayedDeliveryTest.java | 155 +++++++++++++++ .../client/api/TypedMessageBuilder.java | 35 ++++ .../pulsar/client/impl/ProducerImpl.java | 7 +- .../client/impl/TypedMessageBuilderImpl.java | 16 ++ .../pulsar/common/api/proto/PulsarApi.java | 57 ++++++ .../policies/data/SubscriptionStats.java | 5 +- .../collections/TripleLongPriorityQueue.java | 187 ++++++++++++++++++ pulsar-common/src/main/proto/PulsarApi.proto | 5 +- .../TripleLongPriorityQueueTest.java | 52 +++++ .../testclient/PerformanceProducer.java | 31 +-- 24 files changed, 903 insertions(+), 52 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTracker.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueue.java create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueueTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 1bb171c7e7490..9ea0725df9d38 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -245,6 +245,16 @@ maxConsumersPerSubscription=0 # Interval between checks to see if topics with compaction policies need to be compacted brokerServiceCompactionMonitorIntervalInSeconds=60 +# Whether to enable the delayed delivery for messages. +# If disabled, messages will be immediately delivered and there will +# be no tracking overhead. +delayedDeliveryEnabled=true + +# Control the tick time for when retrying on delayed delivery, +# affecting the accuracy of the delivery time compared to the scheduled time. +# Default is 1 second. +delayedDeliveryTickTimeMillis=1000 + ### --- Authentication --- ### # Role names that are treated as "proxy roles". If the broker sees a request with #role as proxyRoles - it will demand to see a valid original principal. 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 b3262c1e59659..94d55c7d4ca5b 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 @@ -154,6 +154,13 @@ public class ServiceConfiguration implements PulsarConfiguration { // waiting for another HTTP call to complete in same thread. private int numHttpServerThreads = Math.max(8, 2 * Runtime.getRuntime().availableProcessors()); + @FieldContext(category = CATEGORY_SERVER, doc = "Whether to enable the delayed delivery for messages.") + private boolean delayedDeliveryEnabled = true; + + @FieldContext(category = CATEGORY_SERVER, doc = "Control the tick time for when retrying on delayed delivery, " + + " affecting the accuracy of the delivery time compared to the scheduled time. Default is 1 second.") + private long delayedDeliveryTickTimeMillis = 1000; + @FieldContext( category = CATEGORY_WEBSOCKET, doc = "Enable the WebSocket API service in broker" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 52c43c40da5cc..0de81db30c44b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -28,6 +28,8 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import io.netty.util.HashedWheelTimer; +import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; @@ -166,6 +168,8 @@ public class PulsarService implements AutoCloseable { private SchemaRegistryService schemaRegistryService = null; private final Optional functionWorkerService; + private Optional delayedDeliveryTimer = Optional.empty(); + private final MessagingServiceShutdownHook shutdownService; private MetricsGenerator metricsGenerator; @@ -214,6 +218,10 @@ public void close() throws PulsarServerException { return; } + if (delayedDeliveryTimer.isPresent()) { + delayedDeliveryTimer.get().stop(); + } + // close the service in reverse order v.s. in which they are started if (this.webService != null) { this.webService.close(); @@ -543,7 +551,7 @@ private void startZkCacheService() throws PulsarServerException { this.localZkCache = new LocalZooKeeperCache(getZkClient(), config.getZooKeeperOperationTimeoutSeconds(), getOrderedExecutor()); this.globalZkCache = new GlobalZooKeeperCache(getZooKeeperClientFactory(), - (int) config.getZooKeeperSessionTimeoutMillis(), + (int) config.getZooKeeperSessionTimeoutMillis(), config.getZooKeeperOperationTimeoutSeconds(), config.getConfigurationStoreServers(), getOrderedExecutor(), this.cacheExecutor); try { @@ -883,7 +891,7 @@ public static String brokerUrlTls(String host, int port) { } public static String webAddress(ServiceConfiguration config) { - if (config.getWebServicePort().isPresent()) { + if (config.getWebServicePort().isPresent()) { return webAddress(advertisedAddress(config), config.getWebServicePort().get()); } else { return null; @@ -942,6 +950,16 @@ public SchemaRegistryService getSchemaRegistryService() { return schemaRegistryService; } + public synchronized Timer getDelayedDeliveryTimer() { + // Lazy initialize + if (!delayedDeliveryTimer.isPresent()) { + delayedDeliveryTimer = Optional.of(new HashedWheelTimer(new DefaultThreadFactory("pulsar-delayed-delivery"), + config.getDelayedDeliveryTickTimeMillis(), TimeUnit.MILLISECONDS)); + } + + return delayedDeliveryTimer.get(); + } + private void startWorkerService(AuthenticationService authenticationService, AuthorizationService authorizationService) throws InterruptedException, IOException, KeeperException { 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 ec0ec66f75e53..9fca8a8674c18 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 @@ -39,8 +39,6 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.stream.Collectors; -import lombok.Data; - import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -56,6 +54,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData; +import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ConsumerStats; @@ -302,6 +301,21 @@ public static int getBatchSizeforEntry(ByteBuf metadataAndPayload, Subscription return -1; } + public static MessageMetadata peekMessageMetadata(ByteBuf metadataAndPayload, Subscription subscription, + long consumerId) { + try { + // save the reader index and restore after parsing + int readerIdx = metadataAndPayload.readerIndex(); + PulsarApi.MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); + metadataAndPayload.readerIndex(readerIdx); + + return metadata; + } catch (Throwable t) { + log.error("[{}] [{}] Failed to parse message metadata", subscription, consumerId, t); + return null; + } + } + void updatePermitsAndPendingAcks(final List entries, SendMessageInfo sentMessages) throws PulsarServerException { int permitsToReduce = 0; Iterator iter = entries.iterator(); @@ -311,15 +325,26 @@ void updatePermitsAndPendingAcks(final List entries, SendMessageInfo sent while (iter.hasNext()) { Entry entry = iter.next(); ByteBuf metadataAndPayload = entry.getDataBuffer(); - int batchSize = getBatchSizeforEntry(metadataAndPayload, subscription, consumerId); - if (batchSize == -1) { - // this would suggest that the message might have been corrupted + MessageMetadata msgMetadata = peekMessageMetadata(metadataAndPayload, subscription, consumerId); + PositionImpl pos = (PositionImpl) entry.getPosition(); + if (msgMetadata == null) { + // Message metadata was corrupted iter.remove(); - PositionImpl pos = (PositionImpl) entry.getPosition(); entry.release(); subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual, Collections.emptyMap()); continue; + } else if (msgMetadata.hasDeliverAtTime() + && subscription.getDispatcher() + .trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { + // The message is marked for delayed delivery. Ignore for now. + iter.remove(); + entry.release(); + continue; } + + int batchSize = msgMetadata.getNumMessagesInBatch(); + msgMetadata.recycle(); + if (pendingAcks != null) { pendingAcks.put(entry.getLedgerId(), entry.getEntryId(), batchSize, 0); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java index f9271c750bbbf..3f839b17197c5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.utils.CopyOnWriteArrayList; @@ -82,4 +83,16 @@ default Optional getRateLimiter() { default void initializeDispatchRateLimiterIfNeeded(Optional policies) { //No-op } + + /** + * Check with dispatcher if the message should be added to the delayed delivery tracker. + * Return true if the message should be delayed and ignored at this point. + */ + default boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata msgMetadata) { + return false; + } + + default long getNumberOfDelayedMessages() { + return 0; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index 8076bd30e6ea5..b8b2793a843dc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -48,6 +48,8 @@ public interface Subscription { long getNumberOfEntriesInBacklog(); + long getNumberOfEntriesDelayed(); + List getConsumers(); CompletableFuture close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTracker.java new file mode 100644 index 0000000000000..3f2709e53b78e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTracker.java @@ -0,0 +1,180 @@ +/** + * 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 io.netty.util.Timeout; +import io.netty.util.Timer; +import io.netty.util.TimerTask; + +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.TimeUnit; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; + +@Slf4j +public class DelayedDeliveryTracker implements AutoCloseable, TimerTask { + + private final TripleLongPriorityQueue priorityQueue = new TripleLongPriorityQueue(); + + private final PersistentDispatcherMultipleConsumers dispatcher; + + // Reference to the shared (per-broker) timer for delayed delivery + private final Timer timer; + + // Current timeout or null if not set + private Timeout timeout; + + // Timestamp at which the timeout is currently set + private long currentTimeoutTarget; + + private final long tickTimeMillis; + + public DelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher) { + this.dispatcher = dispatcher; + this.timer = dispatcher.getTopic().getBrokerService().pulsar().getDelayedDeliveryTimer(); + this.tickTimeMillis = dispatcher.getTopic().getBrokerService().pulsar().getConfiguration() + .getDelayedDeliveryTickTimeMillis(); + } + + public boolean addMessage(long ledgerId, long entryId, long deliveryAt) { + long now = System.currentTimeMillis(); + if (log.isDebugEnabled()) { + log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId, + deliveryAt - now); + } + if (deliveryAt < now) { + // It's already about time to deliver this message + return false; + } + + priorityQueue.add(deliveryAt, ledgerId, entryId); + updateTimer(); + return true; + } + + /** + * Return true if there's at least a message that is scheduled to be delivered already + */ + public boolean hasMessageAvailable() { + return !priorityQueue.isEmpty() && priorityQueue.peekN1() <= System.currentTimeMillis(); + } + + /** + * Get a set of position of messages that have already reached + */ + public Set getScheduledMessages(int maxMessages) { + int n = maxMessages; + Set positions = new TreeSet<>(); + long now = System.currentTimeMillis(); + // Pick all the messages that will be ready within the tick time period. + // This is to avoid keeping rescheduling the timer for each message at + // very short delay + long cutoffTime = now + tickTimeMillis; + + while (n > 0 && !priorityQueue.isEmpty()) { + long timestamp = priorityQueue.peekN1(); + if (timestamp > cutoffTime) { + break; + } + + long ledgerId = priorityQueue.peekN2(); + long entryId = priorityQueue.peekN3(); + positions.add(new PositionImpl(ledgerId, entryId)); + + priorityQueue.pop(); + } + + if (log.isDebugEnabled()) { + log.debug("[{}] Get scheduled messags - found {}", dispatcher.getName(), positions.size()); + } + updateTimer(); + return positions; + } + + public long size() { + return priorityQueue.size(); + } + + private void updateTimer() { + if (priorityQueue.isEmpty()) { + if (timeout != null) { + currentTimeoutTarget = -1; + timeout.cancel(); + timeout = null; + } + return; + } + + long timestamp = priorityQueue.peekN1(); + if (timestamp == currentTimeoutTarget) { + // The timer is already set to the correct target time + return; + } + + if (timeout != null) { + timeout.cancel(); + } + + long delayMillis = timestamp - System.currentTimeMillis(); + if (log.isDebugEnabled()) { + log.debug("[{}] Start timer in {} millis", dispatcher.getName(), delayMillis); + } + + if (delayMillis < 0) { + // There are messages that are already ready to be delivered. If + // the dispatcher is not getting them is because the consumer is + // either not connected or slow. + // We don't need to keep retriggering the timer. When the consumer + // catches up, the dispatcher will do the readMoreEntries() and + // get these messages + return; + } + + currentTimeoutTarget = timestamp; + timeout = timer.newTimeout(this, delayMillis, TimeUnit.MILLISECONDS); + } + + @Override + public void run(Timeout timeout) throws Exception { + if (log.isDebugEnabled()) { + log.info("[{}] Timer triggered", dispatcher.getName()); + } + if (timeout.isCancelled()) { + return; + } + + synchronized (dispatcher) { + currentTimeoutTarget = -1; + timeout = null; + dispatcher.readMoreEntries(); + } + } + + @Override + public void close() { + priorityQueue.close(); + if (timeout != null) { + timeout.cancel(); + } + } +} 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 066e2b6bb3d38..31cb4c751dc5f 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 @@ -21,6 +21,10 @@ import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.Lists; + +import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.Set; @@ -44,11 +48,12 @@ import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Consumer.SendMessageInfo; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.InMemoryRedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; -import org.apache.pulsar.broker.service.InMemoryRedeliveryTracker; import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.util.Codec; @@ -58,9 +63,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.ComparisonChain; -import com.google.common.collect.Lists; - /** */ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers implements Dispatcher, ReadEntriesCallback { @@ -69,9 +71,12 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMu private final ManagedCursor cursor; private CompletableFuture closeFuture = null; - LongPairSet messagesToReplay = new ConcurrentSortedLongPairSet(128, 2); + LongPairSet messagesToRedeliver = new ConcurrentSortedLongPairSet(128, 2); private final RedeliveryTracker redeliveryTracker; + private Optional delayedDeliveryTracker = Optional.empty(); + private final boolean isDelayedDeliveryEnabled; + private boolean havePendingRead = false; private boolean havePendingReplayRead = false; private boolean shouldRewindBeforeReadingOrReplaying = false; @@ -105,6 +110,8 @@ public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCurso this.readBatchSize = serviceConfig.getDispatcherMaxReadBatchSize(); this.maxUnackedMessages = topic.getBrokerService().pulsar().getConfiguration() .getMaxUnackedMessagesPerSubscription(); + this.isDelayedDeliveryEnabled = topic.getBrokerService().pulsar().getConfiguration() + .isDelayedDeliveryEnabled(); this.initializeDispatchRateLimiterIfNeeded(Optional.empty()); } @@ -123,7 +130,7 @@ public synchronized void addConsumer(Consumer consumer) throws BrokerServiceExce cursor.rewind(); shouldRewindBeforeReadingOrReplaying = false; } - messagesToReplay.clear(); + messagesToRedeliver.clear(); } if (isConsumersExceededOnTopic()) { @@ -189,7 +196,7 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE havePendingRead = false; } - messagesToReplay.clear(); + messagesToRedeliver.clear(); if (closeFuture != null) { log.info("[{}] All consumers removed. Subscription is disconnected", name); closeFuture.complete(null); @@ -200,7 +207,7 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE log.debug("[{}] Consumer are left, reading more entries", name); } consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, none) -> { - messagesToReplay.add(ledgerId, entryId); + messagesToRedeliver.add(ledgerId, entryId); }); totalAvailablePermits -= consumer.getAvailablePermits(); readMoreEntries(); @@ -278,14 +285,13 @@ public void readMoreEntries() { } - if (!messagesToReplay.isEmpty()) { + if (hasMessagesToReplay()) { if (havePendingReplayRead) { log.debug("[{}] Skipping replay while awaiting previous read to complete", name); return; } - Set messagesToReplayNow = messagesToReplay.items(messagesToRead, - (ledgerId, entryId) -> new PositionImpl(ledgerId, entryId)); + Set messagesToReplayNow = getMessagesToReplayNow(messagesToRead); if (log.isDebugEnabled()) { log.debug("[{}] Schedule replay of {} messages for {} consumers", name, messagesToReplayNow.size(), @@ -297,9 +303,9 @@ public void readMoreEntries() { ReadType.Replay); // clear already acked positions from replay bucket - deletedMessages.forEach(position -> messagesToReplay.remove(((PositionImpl) position).getLedgerId(), + deletedMessages.forEach(position -> messagesToRedeliver.remove(((PositionImpl) position).getLedgerId(), ((PositionImpl) position).getEntryId())); - // if all the entries are acked-entries and cleared up from messagesToReplay, try to read + // if all the entries are acked-entries and cleared up from messagesToRedeliver, try to read // next entries as readCompletedEntries-callback was never called if ((messagesToReplayNow.size() - deletedMessages.size()) == 0) { havePendingReplayRead = false; @@ -342,8 +348,12 @@ public synchronized boolean canUnsubscribe(Consumer consumer) { } @Override - public CompletableFuture close() { + public synchronized CompletableFuture close() { IS_CLOSED_UPDATER.set(this, TRUE); + if (delayedDeliveryTracker.isPresent()) { + delayedDeliveryTracker.get().close(); + delayedDeliveryTracker = Optional.empty(); + } return disconnectAllConsumers(); } @@ -429,11 +439,12 @@ public synchronized void readEntriesComplete(List entries, Object ctx) { // remove positions first from replay list first : sendMessages recycles entries if (readType == ReadType.Replay) { entries.subList(start, start + messagesForC).forEach(entry -> { - messagesToReplay.remove(entry.getLedgerId(), entry.getEntryId()); + messagesToRedeliver.remove(entry.getLedgerId(), entry.getEntryId()); }); } - SendMessageInfo sentMsgInfo = c.sendMessages(entries.subList(start, start + messagesForC)); + SendMessageInfo sentMsgInfo = c + .sendMessages(new ArrayList<>(entries.subList(start, start + messagesForC))); long msgSent = sentMsgInfo.getTotalSentMessages(); start += messagesForC; @@ -461,7 +472,7 @@ public synchronized void readEntriesComplete(List entries, Object ctx) { entries.size() - start); } entries.subList(start, entries.size()).forEach(entry -> { - messagesToReplay.add(entry.getLedgerId(), entry.getEntryId()); + messagesToRedeliver.add(entry.getLedgerId(), entry.getEntryId()); entry.release(); }); } @@ -503,7 +514,7 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj if (exception instanceof ManagedLedgerException.InvalidReplayPositionException) { PositionImpl markDeletePosition = (PositionImpl) cursor.getMarkDeletedPosition(); - messagesToReplay.removeIf((ledgerId, entryId) -> { + messagesToRedeliver.removeIf((ledgerId, entryId) -> { return ComparisonChain.start().compare(ledgerId, markDeletePosition.getLedgerId()) .compare(entryId, markDeletePosition.getEntryId()).result() <= 0; }); @@ -552,11 +563,11 @@ public boolean isConsumerAvailable(Consumer consumer) { @Override public synchronized void redeliverUnacknowledgedMessages(Consumer consumer) { consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, none) -> { - messagesToReplay.add(ledgerId, entryId); + messagesToRedeliver.add(ledgerId, entryId); }); if (log.isDebugEnabled()) { log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, - messagesToReplay); + messagesToRedeliver); } readMoreEntries(); } @@ -564,7 +575,7 @@ public synchronized void redeliverUnacknowledgedMessages(Consumer consumer) { @Override public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { positions.forEach(position -> { - messagesToReplay.add(position.getLedgerId(), position.getEntryId()); + messagesToRedeliver.add(position.getLedgerId(), position.getEntryId()); redeliveryTracker.incrementAndGetRedeliveryCount(position); }); if (log.isDebugEnabled()) { @@ -643,6 +654,59 @@ public void initializeDispatchRateLimiterIfNeeded(Optional policies) { this.dispatchRateLimiter = Optional.of(new DispatchRateLimiter(topic, name)); } } - + + @Override + public synchronized boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata msgMetadata) { + if (!isDelayedDeliveryEnabled) { + // If broker has the feature disabled, always deliver messages immediately + return false; + } + + if (!delayedDeliveryTracker.isPresent()) { + // Initialize the tracker the first time we need to use it + delayedDeliveryTracker = Optional.of(new DelayedDeliveryTracker(this)); + } + + return delayedDeliveryTracker.get().addMessage(ledgerId, entryId, msgMetadata.getDeliverAtTime()); + } + + /** + * Returns whether we have any message that could be immediately replayed. + * This could be a message that was requested to be re-delivered or a delayed + * delivery. + */ + private boolean hasMessagesToReplay() { + if (!messagesToRedeliver.isEmpty()) { + return true; + } + + if (delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().hasMessageAvailable()) { + return true; + } + + return false; + } + + private Set getMessagesToReplayNow(int maxMessagesToRead) { + if (!messagesToRedeliver.isEmpty()) { + return messagesToRedeliver.items(maxMessagesToRead, + (ledgerId, entryId) -> new PositionImpl(ledgerId, entryId)); + } else { + return delayedDeliveryTracker.get().getScheduledMessages(maxMessagesToRead); + } + } + + public long getNumberOfDelayedMessages() { + if (delayedDeliveryTracker.isPresent()) { + return delayedDeliveryTracker.get().size(); + } else { + return 0; + } + } + + public PersistentTopic getTopic() { + return topic; + } + private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumers.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 669c6d41d457b..0ebbbf7262543 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 @@ -634,10 +634,10 @@ public SubscriptionStats getStats() { } if (SubType.Shared.equals(subStats.type)) { if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { - subStats.unackedMessages = ((PersistentDispatcherMultipleConsumers) dispatcher) - .getTotalUnackedMessages(); - subStats.blockedSubscriptionOnUnackedMsgs = ((PersistentDispatcherMultipleConsumers) dispatcher) - .isBlockedDispatcherOnUnackedMsgs(); + PersistentDispatcherMultipleConsumers d = (PersistentDispatcherMultipleConsumers) dispatcher; + subStats.unackedMessages = d.getTotalUnackedMessages(); + subStats.blockedSubscriptionOnUnackedMsgs = d.isBlockedDispatcherOnUnackedMsgs(); + subStats.msgDelayed = d.getNumberOfDelayedMessages(); } } subStats.msgBacklog = getNumberOfEntriesInBacklog(); @@ -657,10 +657,19 @@ public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List } @Override - public void addUnAckedMessages(int unAckMessages) { + public synchronized void addUnAckedMessages(int unAckMessages) { dispatcher.addUnAckedMessages(unAckMessages); } + @Override + public synchronized long getNumberOfEntriesDelayed() { + if (dispatcher != null) { + return dispatcher.getNumberOfDelayedMessages(); + } else { + return 0; + } + } + @Override public void markTopicWithBatchMessagePublished() { topic.markBatchMessagePublished(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index cb485c2248a2f..04b241f299f1b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -85,6 +85,7 @@ void updateStats(TopicStats stats) { subscriptionStats.computeIfAbsent(n, k -> new AggregatedSubscriptionStats()); subsStats.blockedSubscriptionOnUnackedMsgs = as.blockedSubscriptionOnUnackedMsgs; subsStats.msgBacklog += as.msgBacklog; + subsStats.msgDelayed += as.msgDelayed; subsStats.msgRateRedeliver += as.msgRateRedeliver; subsStats.unackedMessages += as.unackedMessages; as.consumerStat.forEach((c, v) -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java index c46bbf5a2be44..1f3c51302e980 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java @@ -37,5 +37,7 @@ public class AggregatedSubscriptionStats { public double msgThroughputOut; + public long msgDelayed; + public Map consumerStat = new HashMap<>(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 2e7836b453077..54625d56ae682 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -119,6 +119,7 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include AggregatedSubscriptionStats subsStats = stats.subscriptionStats .computeIfAbsent(name, k -> new AggregatedSubscriptionStats()); subsStats.msgBacklog = subscription.getNumberOfEntriesInBacklog(); + subsStats.msgDelayed = subscription.getNumberOfEntriesDelayed(); subscription.getConsumers().forEach(consumer -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index 3549820eb106b..23009f115c41d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -111,6 +111,7 @@ static void printTopicStats(SimpleTextOutputStream stream, String cluster, Strin stats.subscriptionStats.forEach((n, subsStats) -> { metric(stream, cluster, namespace, topic, n, "pulsar_subscription_back_log", subsStats.msgBacklog); + metric(stream, cluster, namespace, topic, n, "pulsar_subscription_delayed", subsStats.msgDelayed); metric(stream, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_redeliver", subsStats.msgRateRedeliver); metric(stream, cluster, namespace, topic, n, "pulsar_subscription_unacked_massages", subsStats.unackedMessages); metric(stream, cluster, namespace, topic, n, "pulsar_subscription_blocked_on_unacked_messages", subsStats.blockedSubscriptionOnUnackedMsgs ? 1 : 0); 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 38fec896f5420..7cd19a12dc17f 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 @@ -1303,7 +1303,7 @@ public void testMessageReplay() throws Exception { PersistentSubscription subRef = topicRef.getSubscription(subName); PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) subRef .getDispatcher(); - Field replayMap = PersistentDispatcherMultipleConsumers.class.getDeclaredField("messagesToReplay"); + Field replayMap = PersistentDispatcherMultipleConsumers.class.getDeclaredField("messagesToRedeliver"); replayMap.setAccessible(true); ConcurrentLongPairSet messagesToReplay = new ConcurrentLongPairSet(64, 1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java new file mode 100644 index 0000000000000..51774ae51973f --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -0,0 +1,155 @@ +/** + * 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.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.TimeUnit; + +import lombok.Cleanup; + +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.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class DelayedDeliveryTest extends ProducerConsumerBase { + + @Override + @BeforeClass + public void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @Override + @AfterClass + public void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testDelayedDelivery() + throws Exception { + String topic = "testNegativeAcks-" + System.nanoTime(); + + @Cleanup + Consumer failoverConsumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("failover-sub") + .subscriptionType(SubscriptionType.Failover) + .subscribe(); + + @Cleanup + Consumer sharedConsumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("shared-sub") + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + + for (int i = 0; i < 10; i++) { + producer.newMessage() + .value("msg-" + i) + .deliverAfter(5, TimeUnit.SECONDS) + .sendAsync(); + } + + producer.flush(); + + // Failover consumer will receive the messages immediately while + // the shared consumer will get them after the delay + Message msg = sharedConsumer.receive(100, TimeUnit.MILLISECONDS); + assertEquals(msg, null); + + for (int i = 0; i < 10; i++) { + msg = failoverConsumer.receive(100, TimeUnit.MILLISECONDS); + assertEquals(msg.getValue(), "msg-" + i); + } + + for (int i = 0; i < 10; i++) { + msg = sharedConsumer.receive(10, TimeUnit.SECONDS); + assertEquals(msg.getValue(), "msg-" + i); + } + } + + @Test + public void testInterleavedMessages() + throws Exception { + String topic = "testInterleavedMessages-" + System.nanoTime(); + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("shared-sub") + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + + for (int i = 0; i < 10; i++) { + // Publish 1 message without delay and 1 with delay + producer.newMessage() + .value("immediate-msg-" + i) + .sendAsync(); + + producer.newMessage() + .value("delayed-msg-" + i) + .deliverAfter(5, TimeUnit.SECONDS) + .sendAsync(); + } + + producer.flush(); + + // Failover consumer will receive the messages immediately while + // the shared consumer will get them after the delay + for (int i = 0; i < 10; i++) { + Message msg = consumer.receive(100, TimeUnit.MILLISECONDS); + assertEquals(msg.getValue(), "immediate-msg-" + i); + consumer.acknowledge(msg); + } + + // Delayed messages might not come in same exact order + Set delayedMessages = new TreeSet<>(); + for (int i = 0; i < 10; i++) { + Message msg = consumer.receive(10, TimeUnit.SECONDS); + delayedMessages.add(msg.getValue()); + consumer.acknowledge(msg); + } + + for (int i = 0; i < 10; i++) { + assertTrue(delayedMessages.contains("delayed-msg-" + i)); + } + } +} diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java index a1e2f2da4f6aa..64798ec121449 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; /** * Message builder that constructs a message to be published through a producer. @@ -175,6 +176,36 @@ public interface TypedMessageBuilder extends Serializable { */ TypedMessageBuilder disableReplication(); + /** + * Deliver the message only at or after the specified absolute timestamp. + *

+ * The timestamp is milliseconds and based on UTC (eg: {@link System#currentTimeMillis()}. + *

+ * Note: messages are only delivered with delay when a consumer is consuming + * through a {@link SubscriptionType#Shared} subscription. With other subscription + * types, the messages will still be delivered immediately. + * + * @param timestamp + * absolute timestamp indicating when the message should be delivered to consumers + * @return the message builder instance + */ + TypedMessageBuilder deliverAt(long timestamp); + + /** + * Request to deliver the message only after the specified relative delay. + *

+ * Note: messages are only delivered with delay when a consumer is consuming + * through a {@link SubscriptionType#Shared} subscription. With other subscription + * types, the messages will still be delivered immediately. + * + * @param delay + * the amount of delay before the message will be delivered + * @param unit + * the time unit for the delay + * @return the message builder instance + */ + TypedMessageBuilder deliverAfter(long delay, TimeUnit unit); + /** * Configure the {@link TypedMessageBuilder} from a config map, as an alternative compared * to call the individual builder methods. @@ -203,6 +234,8 @@ public interface TypedMessageBuilder extends Serializable { * {@link #CONF_SEQUENCE_ID}{@code sequenceId}{@code long}{@link #sequenceId(long)} * {@link #CONF_REPLICATION_CLUSTERS}{@code replicationClusters}{@code List}{@link #replicationClusters(List)} * {@link #CONF_DISABLE_REPLICATION}{@code disableReplication}{@code boolean}{@link #disableReplication()} + * {@link #CONF_DELIVERY_AFTER_SECONDS}{@code deliverAfterSeconds}{@code long}{@link #deliverAfter(long, TimeUnit)} + * {@link #CONF_DELIVERY_AT}{@code deliverAt}{@code long}{@link #deliverAt(long)} * * * @param config a map with the configuration options for the message @@ -216,4 +249,6 @@ public interface TypedMessageBuilder extends Serializable { static final String CONF_SEQUENCE_ID = "sequenceId"; static final String CONF_REPLICATION_CLUSTERS = "replicationClusters"; static final String CONF_DISABLE_REPLICATION = "disableReplication"; + static final String CONF_DELIVERY_AFTER_SECONDS = "deliverAfterSeconds"; + static final String CONF_DELIVERY_AT = "deliverAt"; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index b4a0993b49c8c..868c09f795cdc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -305,8 +305,9 @@ public void sendAsync(Message message, SendCallback callback) { // If compression is enabled, we are compressing, otherwise it will simply use the same buffer int uncompressedSize = payload.readableBytes(); ByteBuf compressedPayload = payload; - // batch will be compressed when closed - if (!isBatchMessagingEnabled()) { + // Batch will be compressed when closed + // If a message has a delayed delivery time, we'll always send it individually + if (!isBatchMessagingEnabled() || msgMetadataBuilder.hasDeliverAtTime()) { compressedPayload = compressor.encode(payload); payload.release(); @@ -361,7 +362,7 @@ public void sendAsync(Message message, SendCallback callback) { msgMetadataBuilder.setUncompressedSize(uncompressedSize); } - if (isBatchMessagingEnabled()) { + if (isBatchMessagingEnabled() && !msgMetadataBuilder.hasDeliverAtTime()) { // handle boundary cases where message being added would exceed // batch size and/or max message size if (batchMessageContainer.hasSpaceInBatch(msg)) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java index fb18c0ec907eb..7b366581e3f30 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; @@ -131,6 +132,17 @@ public TypedMessageBuilder disableReplication() { return this; } + @Override + public TypedMessageBuilder deliverAfter(long delay, TimeUnit unit) { + return deliverAt(System.currentTimeMillis() + unit.toMillis(delay)); + } + + @Override + public TypedMessageBuilder deliverAt(long timestamp) { + msgMetadataBuilder.setDeliverAtTime(timestamp); + return this; + } + @SuppressWarnings("unchecked") @Override public TypedMessageBuilder loadConf(Map config) { @@ -150,6 +162,10 @@ public TypedMessageBuilder loadConf(Map config) { if (disableReplication) { this.disableReplication(); } + } else if (key.equals(CONF_DELIVERY_AFTER_SECONDS)) { + this.deliverAfter(checkType(value, Long.class), TimeUnit.SECONDS); + } else if (key.equals(CONF_DELIVERY_AT)) { + this.deliverAt(checkType(value, Long.class)); } else { throw new RuntimeException("Invalid message config key '" + key + "'"); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java index b2419f1b41c4f..2f02e37c30ed4 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java @@ -3085,6 +3085,10 @@ public interface MessageMetadataOrBuilder // optional bool partition_key_b64_encoded = 17 [default = false]; boolean hasPartitionKeyB64Encoded(); boolean getPartitionKeyB64Encoded(); + + // optional uint64 deliver_at_time = 18; + boolean hasDeliverAtTime(); + long getDeliverAtTime(); } public static final class MessageMetadata extends org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite @@ -3395,6 +3399,16 @@ public boolean getPartitionKeyB64Encoded() { return partitionKeyB64Encoded_; } + // optional uint64 deliver_at_time = 18; + public static final int DELIVER_AT_TIME_FIELD_NUMBER = 18; + private long deliverAtTime_; + public boolean hasDeliverAtTime() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + public long getDeliverAtTime() { + return deliverAtTime_; + } + private void initFields() { producerName_ = ""; sequenceId_ = 0L; @@ -3412,6 +3426,7 @@ private void initFields() { encryptionParam_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; partitionKeyB64Encoded_ = false; + deliverAtTime_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -3502,6 +3517,9 @@ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStr if (((bitField0_ & 0x00001000) == 0x00001000)) { output.writeBool(17, partitionKeyB64Encoded_); } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + output.writeUInt64(18, deliverAtTime_); + } } private int memoizedSerializedSize = -1; @@ -3579,6 +3597,10 @@ public int getSerializedSize() { size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream .computeBoolSize(17, partitionKeyB64Encoded_); } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(18, deliverAtTime_); + } memoizedSerializedSize = size; return size; } @@ -3724,6 +3746,8 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00004000); partitionKeyB64Encoded_ = false; bitField0_ = (bitField0_ & ~0x00008000); + deliverAtTime_ = 0L; + bitField0_ = (bitField0_ & ~0x00010000); return this; } @@ -3825,6 +3849,10 @@ public org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata buildPartial to_bitField0_ |= 0x00001000; } result.partitionKeyB64Encoded_ = partitionKeyB64Encoded_; + if (((from_bitField0_ & 0x00010000) == 0x00010000)) { + to_bitField0_ |= 0x00002000; + } + result.deliverAtTime_ = deliverAtTime_; result.bitField0_ = to_bitField0_; return result; } @@ -3900,6 +3928,9 @@ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.MessageMet if (other.hasPartitionKeyB64Encoded()) { setPartitionKeyB64Encoded(other.getPartitionKeyB64Encoded()); } + if (other.hasDeliverAtTime()) { + setDeliverAtTime(other.getDeliverAtTime()); + } return this; } @@ -4039,6 +4070,11 @@ public Builder mergeFrom( partitionKeyB64Encoded_ = input.readBool(); break; } + case 144: { + bitField0_ |= 0x00010000; + deliverAtTime_ = input.readUInt64(); + break; + } } } } @@ -4621,6 +4657,27 @@ public Builder clearPartitionKeyB64Encoded() { return this; } + // optional uint64 deliver_at_time = 18; + private long deliverAtTime_ ; + public boolean hasDeliverAtTime() { + return ((bitField0_ & 0x00010000) == 0x00010000); + } + public long getDeliverAtTime() { + return deliverAtTime_; + } + public Builder setDeliverAtTime(long value) { + bitField0_ |= 0x00010000; + deliverAtTime_ = value; + + return this; + } + public Builder clearDeliverAtTime() { + bitField0_ = (bitField0_ & ~0x00010000); + deliverAtTime_ = 0L; + + return this; + } + // @@protoc_insertion_point(builder_scope:pulsar.proto.MessageMetadata) } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index f141d4a6d2c1a..c24220c549584 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -43,7 +43,10 @@ public class SubscriptionStats { /** Flag to verify if subscription is blocked due to reaching threshold of unacked messages */ public boolean blockedSubscriptionOnUnackedMsgs; - + + /** Number of delayed messages currently being tracked */ + public long msgDelayed; + /** Number of unacknowledged messages for the subscription */ public long unackedMessages; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueue.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueue.java new file mode 100644 index 0000000000000..af2673fbb5ffb --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueue.java @@ -0,0 +1,187 @@ +/** + * 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.common.util.collections; + +import static com.google.common.base.Preconditions.checkArgument; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; + +/** + * Provides a priority-queue implementation specialized on items composed by 3 longs. + * + * This class is not thread safe and the items are stored in direct memory. + */ +public class TripleLongPriorityQueue implements AutoCloseable { + + private static final int SIZE_OF_LONG = 8; + private static final int DEFAULT_INITIAL_CAPACITY = 16; + + // Each item is composed of 3 longs + private static final int ITEMS_COUNT = 3; + + private static final int TUPLE_SIZE = ITEMS_COUNT * SIZE_OF_LONG; + + private final ByteBuf buffer; + + private int capacity; + private int size; + + public TripleLongPriorityQueue() { + this(DEFAULT_INITIAL_CAPACITY); + } + + public TripleLongPriorityQueue(int initialCapacity) { + capacity = initialCapacity; + buffer = PooledByteBufAllocator.DEFAULT.directBuffer(initialCapacity * ITEMS_COUNT * SIZE_OF_LONG); + size = 0; + } + + @Override + public void close() { + buffer.release(); + } + + public void add(long n1, long n2, long n3) { + if (size == capacity) { + increaseCapacity(); + } + + put(size, n1, n2, n3); + siftUp(size); + ++size; + } + + public long peekN1() { + checkArgument(size != 0); + return buffer.getLong(0); + } + + public long peekN2() { + checkArgument(size != 0); + return buffer.getLong(0 + 1 * SIZE_OF_LONG); + } + + public long peekN3() { + checkArgument(size != 0); + return buffer.getLong(0 + 2 * SIZE_OF_LONG); + } + + /** + * Removes the first item from the queue. + */ + public void pop() { + checkArgument(size != 0); + swap(0, size - 1); + size--; + siftDown(0); + } + + public boolean isEmpty() { + return size == 0; + } + + public int size() { + return size; + } + + private void increaseCapacity() { + // For bigger sizes, increase by 50% + this.capacity += (capacity <= 256 ? capacity : capacity / 2); + buffer.capacity(this.capacity * TUPLE_SIZE); + } + + private void siftUp(int idx) { + while (idx > 0) { + int parentIdx = (idx - 1) / 2; + if (compare(idx, parentIdx) >= 0) { + break; + } + + swap(idx, parentIdx); + idx = parentIdx; + } + } + + private void siftDown(int idx) { + int half = size / 2; + while (idx < half) { + int left = 2 * idx + 1; + int right = 2 * idx + 2; + + int swapIdx = idx; + + if (compare(idx, left) > 0) { + swapIdx = left; + } + + if (right < size && compare(swapIdx, right) > 0) { + swapIdx = right; + } + + if (swapIdx == idx) { + return; + } + + swap(idx, swapIdx); + idx = swapIdx; + } + } + + private void put(int idx, long n1, long n2, long n3) { + int i = idx * TUPLE_SIZE; + buffer.setLong(i, n1); + buffer.setLong(i + 1 * SIZE_OF_LONG, n2); + buffer.setLong(i + 2 * SIZE_OF_LONG, n3); + } + + private int compare(int idx1, int idx2) { + int i1 = idx1 * TUPLE_SIZE; + int i2 = idx2 * TUPLE_SIZE; + + int c1 = Long.compare(buffer.getLong(i1), buffer.getLong(i2)); + if (c1 != 0) { + return c1; + } + + int c2 = Long.compare(buffer.getLong(i1 + SIZE_OF_LONG), buffer.getLong(i2 + SIZE_OF_LONG)); + if (c2 != 0) { + return c2; + } + + return Long.compare(buffer.getLong(i1 + 2 * SIZE_OF_LONG), buffer.getLong(i2 + 2 * SIZE_OF_LONG)); + } + + private void swap(int idx1, int idx2) { + int i1 = idx1 * TUPLE_SIZE; + int i2 = idx2 * TUPLE_SIZE; + + long tmp1 = buffer.getLong(i1); + long tmp2 = buffer.getLong(i1 + 1 * SIZE_OF_LONG); + long tmp3 = buffer.getLong(i1 + 2 * SIZE_OF_LONG); + + buffer.setLong(i1, buffer.getLong(i2)); + buffer.setLong(i1 + 1 * SIZE_OF_LONG, buffer.getLong(i2 + 1 * SIZE_OF_LONG)); + buffer.setLong(i1 + 2 * SIZE_OF_LONG, buffer.getLong(i2 + 2 * SIZE_OF_LONG)); + + buffer.setLong(i2, tmp1); + buffer.setLong(i2 + 1 * SIZE_OF_LONG, tmp2); + buffer.setLong(i2 + 2 * SIZE_OF_LONG, tmp3); + } +} diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index c3e9fa2af342b..e222592dffb89 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -111,7 +111,10 @@ message MessageMetadata { optional bytes encryption_param = 15; optional bytes schema_version = 16; - optional bool partition_key_b64_encoded = 17 [ default = false ]; + optional bool partition_key_b64_encoded = 17 [ default = false ]; + + // Mark the message to be delivered at or after the specified timestamp + optional uint64 deliver_at_time = 18; } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueueTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueueTest.java new file mode 100644 index 0000000000000..9829c424ed05f --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueueTest.java @@ -0,0 +1,52 @@ +/** + * 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.common.util.collections; + +import static org.testng.Assert.assertEquals; + +import org.testng.annotations.Test; + +public class TripleLongPriorityQueueTest { + + @Test + public void testQueue() { + TripleLongPriorityQueue pq = new TripleLongPriorityQueue(); + assertEquals(pq.size(), 0); + + final int N = 20; + + for (int i = N; i > 0; i--) { + pq.add(i, i * 2, i * 3); + } + + assertEquals(pq.size(), N); + + for (int i = 1; i <= N; i++) { + assertEquals(pq.peekN1(), i); + assertEquals(pq.peekN2(), i * 2); + assertEquals(pq.peekN3(), i * 3); + + pq.pop(); + + assertEquals(pq.size(), N - i); + } + + pq.close(); + } +} diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java index 440ebf75c6c8f..0febc666c2b66 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java @@ -22,6 +22,16 @@ import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.RateLimiter; + +import io.netty.util.concurrent.DefaultThreadFactory; + import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.PrintStream; @@ -50,20 +60,11 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.testclient.utils.PaddingDecimalFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; -import com.beust.jcommander.ParameterException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectWriter; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.RateLimiter; - -import io.netty.util.concurrent.DefaultThreadFactory; - public class PerformanceProducer { private static final ExecutorService executor = Executors @@ -153,6 +154,9 @@ static class Arguments { "--encryption-key-value-file" }, description = "The file which contains the public key to encrypt payload") public String encKeyFile = null; + @Parameter(names = { "-d", + "--delay" }, description = "Mark messages with a given delay in seconds") + public long delay = 0; } public static void main(String[] args) throws Exception { @@ -345,7 +349,12 @@ public void run() { final long sendTime = System.nanoTime(); - producer.sendAsync(payloadData).thenRun(() -> { + TypedMessageBuilder messageBuilder = producer.newMessage() + .value(payloadData); + if (arguments.delay >0) { + messageBuilder.deliverAfter(arguments.delay, TimeUnit.SECONDS); + } + messageBuilder.sendAsync().thenRun(() -> { messagesSent.increment(); bytesSent.add(payloadData.length); From b54c4381c9115afdece82050fa6a19a2330dc581 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 17 Apr 2019 13:32:08 -0700 Subject: [PATCH 02/15] Fixed compilation --- .../java/org/apache/pulsar/broker/service/Subscription.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index b8b2793a843dc..011d65d455d1e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -48,7 +48,9 @@ public interface Subscription { long getNumberOfEntriesInBacklog(); - long getNumberOfEntriesDelayed(); + default long getNumberOfEntriesDelayed() { + return 0; + } List getConsumers(); From d5e1ebc628259bdf3c1867f038cb8aed061164ab Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 18 Apr 2019 10:01:04 -0700 Subject: [PATCH 03/15] Allow to configure the delayed tracker implementation --- .../pulsar/broker/ServiceConfiguration.java | 3 + .../apache/pulsar/broker/PulsarService.java | 25 ++----- .../delayed/DelayedDeliveryTracker.java | 67 +++++++++++++++++++ .../DelayedDeliveryTrackerFactory.java | 55 +++++++++++++++ .../delayed/DelayedDeliveryTrackerLoader.java | 49 ++++++++++++++ .../InMemoryDelayedDeliveryTracker.java} | 18 +++-- ...InMemoryDelayedDeliveryTrackerFactory.java | 55 +++++++++++++++ .../broker/loadbalance/LoadManager.java | 10 ++- .../pulsar/broker/service/BrokerService.java | 13 ++++ .../pulsar/broker/service/Consumer.java | 18 ----- ...PersistentDispatcherMultipleConsumers.java | 7 +- ...sistentDispatcherSingleActiveConsumer.java | 6 +- ...PersistentDispatcherMultipleConsumers.java | 5 +- 13 files changed, 273 insertions(+), 58 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java rename pulsar-broker/src/main/java/org/apache/pulsar/broker/{service/persistent/DelayedDeliveryTracker.java => delayed/InMemoryDelayedDeliveryTracker.java} (91%) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java 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 94d55c7d4ca5b..532d912eb9c44 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 @@ -157,6 +157,9 @@ public class ServiceConfiguration implements PulsarConfiguration { @FieldContext(category = CATEGORY_SERVER, doc = "Whether to enable the delayed delivery for messages.") private boolean delayedDeliveryEnabled = true; + @FieldContext(category = CATEGORY_SERVER, doc = "Class name of the factory that implements the delayed deliver tracker") + private String delayedDeliveryTrackerFactoryClassName = "org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTrackerFactory"; + @FieldContext(category = CATEGORY_SERVER, doc = "Control the tick time for when retrying on delayed delivery, " + " affecting the accuracy of the delivery time compared to the scheduled time. Default is 1 second.") private long delayedDeliveryTickTimeMillis = 1000; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 0de81db30c44b..934ce109f0939 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -28,13 +28,14 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import io.netty.util.HashedWheelTimer; -import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; import java.net.URI; -import java.util.*; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -97,9 +98,9 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.TwoPhaseCompactor; -import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; +import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.websocket.WebSocketConsumerServlet; import org.apache.pulsar.websocket.WebSocketProducerServlet; import org.apache.pulsar.websocket.WebSocketReaderServlet; @@ -168,8 +169,6 @@ public class PulsarService implements AutoCloseable { private SchemaRegistryService schemaRegistryService = null; private final Optional functionWorkerService; - private Optional delayedDeliveryTimer = Optional.empty(); - private final MessagingServiceShutdownHook shutdownService; private MetricsGenerator metricsGenerator; @@ -218,10 +217,6 @@ public void close() throws PulsarServerException { return; } - if (delayedDeliveryTimer.isPresent()) { - delayedDeliveryTimer.get().stop(); - } - // close the service in reverse order v.s. in which they are started if (this.webService != null) { this.webService.close(); @@ -950,16 +945,6 @@ public SchemaRegistryService getSchemaRegistryService() { return schemaRegistryService; } - public synchronized Timer getDelayedDeliveryTimer() { - // Lazy initialize - if (!delayedDeliveryTimer.isPresent()) { - delayedDeliveryTimer = Optional.of(new HashedWheelTimer(new DefaultThreadFactory("pulsar-delayed-delivery"), - config.getDelayedDeliveryTickTimeMillis(), TimeUnit.MILLISECONDS)); - } - - return delayedDeliveryTimer.get(); - } - private void startWorkerService(AuthenticationService authenticationService, AuthorizationService authorizationService) throws InterruptedException, IOException, KeeperException { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java new file mode 100644 index 0000000000000..836a2ff57d528 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.delayed; + +import com.google.common.annotations.Beta; + +import java.util.Set; + +import org.apache.bookkeeper.mledger.impl.PositionImpl; + +/** + * Represent the tracker for the delayed delivery of messages for a particular subscription. + * + * Note: this interface is still being refined and some breaking changes might be introduced. + */ +@Beta +public interface DelayedDeliveryTracker extends AutoCloseable { + + /** + * Add a message to the tracker + * + * @param ledgerId + * the ledgerId + * @param entryId + * the entryId + * @param deliveryAt + * the absolute timestamp at which the message should be tracked + * @return true if the message was added to the tracker or false if it should be delivered immediately + */ + boolean addMessage(long ledgerId, long entryId, long deliveryAt); + + /** + * Return true if there's at least a message that is scheduled to be delivered already + */ + boolean hasMessageAvailable(); + + /** + * @return the number of delayed messages being tracked + */ + long getNumberOfDelayedMessages(); + + /** + * Get a set of position of messages that have already reached the delivery time + */ + Set getScheduledMessages(int maxMessages); + + /** + * Close the subscription tracker and release all resources. + */ + void close(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java new file mode 100644 index 0000000000000..b0a16a571e10f --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.delayed; + +import com.google.common.annotations.Beta; + +import java.io.IOException; + +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; + +/** + * Factory of InMemoryDelayedDeliveryTracker objects. This is the entry point for implementations. + * + * Note: this interface is still being refined and some breaking changes might be introduced. + */ +@Beta +public interface DelayedDeliveryTrackerFactory extends AutoCloseable { + /** + * Initialize the factory implementation from the broker service configuration + * + * @param config + * the broker service config object + */ + void initialize(ServiceConfiguration config) throws IOException; + + /** + * Create a new tracker instance. + * + * @param dispatcher + * a multi-consumer dispatcher instance + */ + DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher); + + /** + * Close the factory and release all the resources + */ + void close() throws IOException; +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java new file mode 100644 index 0000000000000..db4cfe6426329 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.delayed; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.io.IOException; + +import lombok.experimental.UtilityClass; + +import org.apache.pulsar.broker.ServiceConfiguration; + +@UtilityClass +public class DelayedDeliveryTrackerLoader { + public static DelayedDeliveryTrackerFactory loadDelayedDeliveryTrackerFactory(ServiceConfiguration conf) + throws IOException { + // try { + Class factoryClass; + try { + factoryClass = Class.forName(conf.getDelayedDeliveryTrackerFactoryClassName()); + Object obj = factoryClass.newInstance(); + checkArgument(obj instanceof DelayedDeliveryTrackerFactory, + "The factory has to be an instance of " + DelayedDeliveryTrackerFactory.class.getName()); + + DelayedDeliveryTrackerFactory factory = (DelayedDeliveryTrackerFactory) obj; + factory.initialize(conf); + return factory; + + } catch (Exception e) { + throw new IOException(e); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java similarity index 91% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTracker.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index 3f2709e53b78e..2dd6ff794e041 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.service.persistent; +package org.apache.pulsar.broker.delayed; import io.netty.util.Timeout; import io.netty.util.Timer; @@ -29,10 +29,11 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; @Slf4j -public class DelayedDeliveryTracker implements AutoCloseable, TimerTask { +public class InMemoryDelayedDeliveryTracker implements DelayedDeliveryTracker, TimerTask { private final TripleLongPriorityQueue priorityQueue = new TripleLongPriorityQueue(); @@ -49,13 +50,13 @@ public class DelayedDeliveryTracker implements AutoCloseable, TimerTask { private final long tickTimeMillis; - public DelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher) { + InMemoryDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis) { this.dispatcher = dispatcher; - this.timer = dispatcher.getTopic().getBrokerService().pulsar().getDelayedDeliveryTimer(); - this.tickTimeMillis = dispatcher.getTopic().getBrokerService().pulsar().getConfiguration() - .getDelayedDeliveryTickTimeMillis(); + this.timer = timer; + this.tickTimeMillis = tickTimeMillis; } + @Override public boolean addMessage(long ledgerId, long entryId, long deliveryAt) { long now = System.currentTimeMillis(); if (log.isDebugEnabled()) { @@ -75,6 +76,7 @@ public boolean addMessage(long ledgerId, long entryId, long deliveryAt) { /** * Return true if there's at least a message that is scheduled to be delivered already */ + @Override public boolean hasMessageAvailable() { return !priorityQueue.isEmpty() && priorityQueue.peekN1() <= System.currentTimeMillis(); } @@ -82,6 +84,7 @@ public boolean hasMessageAvailable() { /** * Get a set of position of messages that have already reached */ + @Override public Set getScheduledMessages(int maxMessages) { int n = maxMessages; Set positions = new TreeSet<>(); @@ -111,7 +114,8 @@ public Set getScheduledMessages(int maxMessages) { return positions; } - public long size() { + @Override + public long getNumberOfDelayedMessages() { return priorityQueue.size(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java new file mode 100644 index 0000000000000..71f0fc293d2e1 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.delayed; + +import io.netty.util.HashedWheelTimer; +import io.netty.util.Timer; +import io.netty.util.concurrent.DefaultThreadFactory; + +import java.util.concurrent.TimeUnit; + +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; + +public class InMemoryDelayedDeliveryTrackerFactory implements DelayedDeliveryTrackerFactory { + + private Timer timer; + + private long tickTimeMillis; + + @Override + public void initialize(ServiceConfiguration config) { + this.timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-delayed-delivery"), + config.getDelayedDeliveryTickTimeMillis(), TimeUnit.MILLISECONDS); + this.tickTimeMillis = config.getDelayedDeliveryTickTimeMillis(); + } + + @Override + public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { + return new InMemoryDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis); + } + + @Override + public void close() { + if (timer != null) { + timer.stop(); + } + } + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java index 6a9dc38f33256..b6493e13a49b3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java @@ -108,12 +108,12 @@ public interface LoadManager { * @throws Exception */ public void disableBroker() throws Exception; - + /** * Get list of available brokers in cluster - * + * * @return - * @throws Exception + * @throws Exception */ Set getAvailableBrokers() throws Exception; @@ -134,9 +134,7 @@ static LoadManager create(final PulsarService pulsar) { // Assume there is a constructor with one argument of PulsarService. final Object loadManagerInstance = loadManagerClass.newInstance(); if (loadManagerInstance instanceof LoadManager) { - final LoadManager casted = (LoadManager) loadManagerInstance; - casted.initialize(pulsar); - return casted; + return (LoadManager) loadManagerInstance; } else if (loadManagerInstance instanceof ModularLoadManager) { final LoadManager casted = new ModularLoadManagerWrapper((ModularLoadManager) loadManagerInstance); casted.initialize(pulsar); 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 fddda96351f72..0f6f5bd66c6e4 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 @@ -80,6 +80,8 @@ import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; +import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerLoader; import org.apache.pulsar.broker.loadbalance.LoadManager; import org.apache.pulsar.broker.service.BrokerServiceException.NamingException; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; @@ -189,6 +191,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener blockedDispatchers; private final ReadWriteLock lock = new ReentrantReadWriteLock(); + private final DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory; + public BrokerService(PulsarService pulsar) throws Exception { this.pulsar = pulsar; this.managedLedgerFactory = pulsar.getManagedLedgerFactory(); @@ -271,9 +275,13 @@ public void recordLatency(EventType eventType, long latencyMs) { pulsarStats.recordZkLatencyTimeValue(eventType, latencyMs); } }; + + this.delayedDeliveryTrackerFactory = DelayedDeliveryTrackerLoader + .loadDelayedDeliveryTrackerFactory(pulsar.getConfiguration()); } public void start() throws Exception { + this.delayedDeliveryTrackerFactory.initialize(pulsar.getConfiguration()); this.producerNameGenerator = new DistributedIdGenerator(pulsar.getZkClient(), producerNameGeneratorPath, pulsar.getConfiguration().getClusterName()); @@ -413,6 +421,7 @@ public void close() throws IOException { ClientCnxnAspect.removeListener(zkStatsListener); ClientCnxnAspect.registerExecutor(null); topicOrderedExecutor.shutdown(); + delayedDeliveryTrackerFactory.close(); log.info("Broker service completely shut down"); } @@ -1375,6 +1384,10 @@ public void onUpdate(String path, Map data, Stat stat) { } + public DelayedDeliveryTrackerFactory getDelayedDeliveryTrackerFactory() { + return delayedDeliveryTrackerFactory; + } + public static List getDynamicConfiguration() { return dynamicConfigurationMap.keys(); } 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 9fca8a8674c18..a62b2c5b416b9 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 @@ -283,24 +283,6 @@ private void incrementUnackedMessages(int ackedMessages) { } } - public static int getBatchSizeforEntry(ByteBuf metadataAndPayload, Subscription subscription, long consumerId) { - try { - // save the reader index and restore after parsing - metadataAndPayload.markReaderIndex(); - PulsarApi.MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); - metadataAndPayload.resetReaderIndex(); - int batchSize = metadata.getNumMessagesInBatch(); - metadata.recycle(); - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] num messages in batch are {} ", subscription, consumerId, batchSize); - } - return batchSize; - } catch (Throwable t) { - log.error("[{}] [{}] Failed to parse message metadata", subscription, consumerId, t); - } - return -1; - } - public static MessageMetadata peekMessageMetadata(ByteBuf metadataAndPayload, Subscription subscription, long consumerId) { try { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java index 2067a80d8efc9..bf8c256534655 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java @@ -18,8 +18,6 @@ */ package org.apache.pulsar.broker.service.nonpersistent; -import static org.apache.pulsar.broker.service.Consumer.getBatchSizeforEntry; - import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -35,6 +33,7 @@ import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; import org.apache.pulsar.utils.CopyOnWriteArrayList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -194,7 +193,9 @@ public void sendMessages(List entries) { TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -consumer.sendMessages(entries).getTotalSentMessages()); } else { entries.forEach(entry -> { - int totalMsgs = getBatchSizeforEntry(entry.getDataBuffer(), subscription, -1); + MessageMetadata msgMetatada = Consumer.peekMessageMetadata(entry.getDataBuffer(), subscription, -1); + int totalMsgs = msgMetatada.getNumMessagesInBatch(); + msgMetatada.recycle(); if (totalMsgs > 0) { msgDrop.recordEvent(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java index 787fb00a940ed..4cdf91d8cadf3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.service.nonpersistent; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import static org.apache.pulsar.broker.service.Consumer.getBatchSizeforEntry; import java.util.List; @@ -32,6 +31,7 @@ import org.apache.pulsar.broker.service.RedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.Policies; @@ -61,7 +61,9 @@ public void sendMessages(List entries) { currentConsumer.sendMessages(entries); } else { entries.forEach(entry -> { - int totalMsgs = getBatchSizeforEntry(entry.getDataBuffer(), subscription, -1); + MessageMetadata msgMetatada = Consumer.peekMessageMetadata(entry.getDataBuffer(), subscription, -1); + int totalMsgs = msgMetatada.getNumMessagesInBatch(); + msgMetatada.recycle(); if (totalMsgs > 0) { msgDrop.recordEvent(); } 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 31cb4c751dc5f..6001ef3c03b14 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 @@ -42,6 +42,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.admin.AdminResource; +import org.apache.pulsar.broker.delayed.DelayedDeliveryTracker; import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; @@ -664,7 +665,7 @@ public synchronized boolean trackDelayedDelivery(long ledgerId, long entryId, Me if (!delayedDeliveryTracker.isPresent()) { // Initialize the tracker the first time we need to use it - delayedDeliveryTracker = Optional.of(new DelayedDeliveryTracker(this)); + delayedDeliveryTracker = Optional.of(topic.getBrokerService().getDelayedDeliveryTrackerFactory().newTracker(this)); } return delayedDeliveryTracker.get().addMessage(ledgerId, entryId, msgMetadata.getDeliverAtTime()); @@ -698,7 +699,7 @@ private Set getMessagesToReplayNow(int maxMessagesToRead) { public long getNumberOfDelayedMessages() { if (delayedDeliveryTracker.isPresent()) { - return delayedDeliveryTracker.get().size(); + return delayedDeliveryTracker.get().getNumberOfDelayedMessages(); } else { return 0; } From 52832feda33e5c798ca7dc9189477cab14340fb9 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 18 Apr 2019 11:28:55 -0700 Subject: [PATCH 04/15] Use int64 for timestamp --- .../PersistentDispatcherMultipleConsumers.java | 4 ++-- .../apache/pulsar/common/api/proto/PulsarApi.java | 12 ++++++------ pulsar-common/src/main/proto/PulsarApi.proto | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) 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 6001ef3c03b14..f6de52ca5c61a 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 @@ -688,7 +688,7 @@ private boolean hasMessagesToReplay() { return false; } - private Set getMessagesToReplayNow(int maxMessagesToRead) { + private synchronized Set getMessagesToReplayNow(int maxMessagesToRead) { if (!messagesToRedeliver.isEmpty()) { return messagesToRedeliver.items(maxMessagesToRead, (ledgerId, entryId) -> new PositionImpl(ledgerId, entryId)); @@ -697,7 +697,7 @@ private Set getMessagesToReplayNow(int maxMessagesToRead) { } } - public long getNumberOfDelayedMessages() { + public synchronized long getNumberOfDelayedMessages() { if (delayedDeliveryTracker.isPresent()) { return delayedDeliveryTracker.get().getNumberOfDelayedMessages(); } else { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java index 2f02e37c30ed4..9199b4ee7cfde 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java @@ -3086,7 +3086,7 @@ public interface MessageMetadataOrBuilder boolean hasPartitionKeyB64Encoded(); boolean getPartitionKeyB64Encoded(); - // optional uint64 deliver_at_time = 18; + // optional int64 deliver_at_time = 18; boolean hasDeliverAtTime(); long getDeliverAtTime(); } @@ -3399,7 +3399,7 @@ public boolean getPartitionKeyB64Encoded() { return partitionKeyB64Encoded_; } - // optional uint64 deliver_at_time = 18; + // optional int64 deliver_at_time = 18; public static final int DELIVER_AT_TIME_FIELD_NUMBER = 18; private long deliverAtTime_; public boolean hasDeliverAtTime() { @@ -3518,7 +3518,7 @@ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStr output.writeBool(17, partitionKeyB64Encoded_); } if (((bitField0_ & 0x00002000) == 0x00002000)) { - output.writeUInt64(18, deliverAtTime_); + output.writeInt64(18, deliverAtTime_); } } @@ -3599,7 +3599,7 @@ public int getSerializedSize() { } if (((bitField0_ & 0x00002000) == 0x00002000)) { size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(18, deliverAtTime_); + .computeInt64Size(18, deliverAtTime_); } memoizedSerializedSize = size; return size; @@ -4072,7 +4072,7 @@ public Builder mergeFrom( } case 144: { bitField0_ |= 0x00010000; - deliverAtTime_ = input.readUInt64(); + deliverAtTime_ = input.readInt64(); break; } } @@ -4657,7 +4657,7 @@ public Builder clearPartitionKeyB64Encoded() { return this; } - // optional uint64 deliver_at_time = 18; + // optional int64 deliver_at_time = 18; private long deliverAtTime_ ; public boolean hasDeliverAtTime() { return ((bitField0_ & 0x00010000) == 0x00010000); diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index e222592dffb89..b870a0fe07bac 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -114,7 +114,7 @@ message MessageMetadata { optional bool partition_key_b64_encoded = 17 [ default = false ]; // Mark the message to be delivered at or after the specified timestamp - optional uint64 deliver_at_time = 18; + optional int64 deliver_at_time = 18; } From 31db950d6da4fc9d5ddf6e6a98d10b8f4236e28b Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 26 Apr 2019 09:31:01 -0700 Subject: [PATCH 05/15] Address comments --- .../delayed/DelayedDeliveryTrackerLoader.java | 1 - .../InMemoryDelayedDeliveryTracker.java | 17 +- .../broker/loadbalance/LoadManager.java | 10 +- .../pulsar/broker/service/BrokerService.java | 1 - .../pulsar/broker/service/Consumer.java | 53 ++++-- ...PersistentDispatcherMultipleConsumers.java | 5 +- ...sistentDispatcherSingleActiveConsumer.java | 5 +- .../delayed/InMemoryDeliveryTrackerTest.java | 156 ++++++++++++++++++ 8 files changed, 213 insertions(+), 35 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java index db4cfe6426329..b74adcf339f55 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java @@ -41,7 +41,6 @@ public static DelayedDeliveryTrackerFactory loadDelayedDeliveryTrackerFactory(Se DelayedDeliveryTrackerFactory factory = (DelayedDeliveryTrackerFactory) obj; factory.initialize(conf); return factory; - } catch (Exception e) { throw new IOException(e); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index 2dd6ff794e041..c692c3ac77ae7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -22,6 +22,7 @@ import io.netty.util.Timer; import io.netty.util.TimerTask; +import java.time.Clock; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.TimeUnit; @@ -50,15 +51,22 @@ public class InMemoryDelayedDeliveryTracker implements DelayedDeliveryTracker, T private final long tickTimeMillis; + private final Clock clock; + InMemoryDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis) { + this(dispatcher, timer, tickTimeMillis, Clock.systemUTC()); + } + + InMemoryDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, Clock clock) { this.dispatcher = dispatcher; this.timer = timer; this.tickTimeMillis = tickTimeMillis; + this.clock = clock; } @Override public boolean addMessage(long ledgerId, long entryId, long deliveryAt) { - long now = System.currentTimeMillis(); + long now = clock.millis(); if (log.isDebugEnabled()) { log.debug("[{}] Add message {}:{} -- Delivery in {} ms ", dispatcher.getName(), ledgerId, entryId, deliveryAt - now); @@ -78,7 +86,7 @@ public boolean addMessage(long ledgerId, long entryId, long deliveryAt) { */ @Override public boolean hasMessageAvailable() { - return !priorityQueue.isEmpty() && priorityQueue.peekN1() <= System.currentTimeMillis(); + return !priorityQueue.isEmpty() && priorityQueue.peekN1() <= clock.millis(); } /** @@ -88,7 +96,7 @@ public boolean hasMessageAvailable() { public Set getScheduledMessages(int maxMessages) { int n = maxMessages; Set positions = new TreeSet<>(); - long now = System.currentTimeMillis(); + long now = clock.millis(); // Pick all the messages that will be ready within the tick time period. // This is to avoid keeping rescheduling the timer for each message at // very short delay @@ -105,6 +113,7 @@ public Set getScheduledMessages(int maxMessages) { positions.add(new PositionImpl(ledgerId, entryId)); priorityQueue.pop(); + --n; } if (log.isDebugEnabled()) { @@ -139,7 +148,7 @@ private void updateTimer() { timeout.cancel(); } - long delayMillis = timestamp - System.currentTimeMillis(); + long delayMillis = timestamp - clock.millis(); if (log.isDebugEnabled()) { log.debug("[{}] Start timer in {} millis", dispatcher.getName(), delayMillis); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java index b6493e13a49b3..6a9dc38f33256 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java @@ -108,12 +108,12 @@ public interface LoadManager { * @throws Exception */ public void disableBroker() throws Exception; - + /** * Get list of available brokers in cluster - * + * * @return - * @throws Exception + * @throws Exception */ Set getAvailableBrokers() throws Exception; @@ -134,7 +134,9 @@ static LoadManager create(final PulsarService pulsar) { // Assume there is a constructor with one argument of PulsarService. final Object loadManagerInstance = loadManagerClass.newInstance(); if (loadManagerInstance instanceof LoadManager) { - return (LoadManager) loadManagerInstance; + final LoadManager casted = (LoadManager) loadManagerInstance; + casted.initialize(pulsar); + return casted; } else if (loadManagerInstance instanceof ModularLoadManager) { final LoadManager casted = new ModularLoadManagerWrapper((ModularLoadManager) loadManagerInstance); casted.initialize(pulsar); 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 0f6f5bd66c6e4..62de76346f1cb 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 @@ -281,7 +281,6 @@ public void recordLatency(EventType eventType, long latencyMs) { } public void start() throws Exception { - this.delayedDeliveryTrackerFactory.initialize(pulsar.getConfiguration()); this.producerNameGenerator = new DistributedIdGenerator(pulsar.getZkClient(), producerNameGeneratorPath, pulsar.getConfiguration().getClusterName()); 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 24fb25ecd8b2e..a09d6acfb59ee 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 @@ -283,6 +283,18 @@ private void incrementUnackedMessages(int ackedMessages) { } } + public static int getNumberOfMessagesInBatch(ByteBuf metadataAndPayload, Subscription subscription, + long consumerId) { + MessageMetadata msgMetadata = peekMessageMetadata(metadataAndPayload, subscription, consumerId); + if (msgMetadata == null) { + return -1; + } else { + int numMessagesInBatch = msgMetadata.getNumMessagesInBatch(); + msgMetadata.recycle(); + return numMessagesInBatch; + } + } + public static MessageMetadata peekMessageMetadata(ByteBuf metadataAndPayload, Subscription subscription, long consumerId) { try { @@ -307,25 +319,32 @@ void updatePermitsAndPendingAcks(final List entries, SendMessageInfo sent while (iter.hasNext()) { Entry entry = iter.next(); ByteBuf metadataAndPayload = entry.getDataBuffer(); - MessageMetadata msgMetadata = peekMessageMetadata(metadataAndPayload, subscription, consumerId); PositionImpl pos = (PositionImpl) entry.getPosition(); - if (msgMetadata == null) { - // Message metadata was corrupted - iter.remove(); - entry.release(); - subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual, Collections.emptyMap()); - continue; - } else if (msgMetadata.hasDeliverAtTime() - && subscription.getDispatcher() - .trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { - // The message is marked for delayed delivery. Ignore for now. - iter.remove(); - entry.release(); - continue; - } - int batchSize = msgMetadata.getNumMessagesInBatch(); - msgMetadata.recycle(); + int batchSize; + MessageMetadata msgMetadata = peekMessageMetadata(metadataAndPayload, subscription, consumerId); + + try { + if (msgMetadata == null) { + // Message metadata was corrupted + iter.remove(); + entry.release(); + subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual, + Collections.emptyMap()); + continue; + } else if (msgMetadata.hasDeliverAtTime() + && subscription.getDispatcher() + .trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { + // The message is marked for delayed delivery. Ignore for now. + iter.remove(); + entry.release(); + continue; + } + + batchSize = msgMetadata.getNumMessagesInBatch(); + } finally { + msgMetadata.recycle(); + } if (pendingAcks != null) { pendingAcks.put(entry.getLedgerId(), entry.getEntryId(), batchSize, 0); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java index bf8c256534655..1e8bbeea26e4b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java @@ -33,7 +33,6 @@ import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; -import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; import org.apache.pulsar.utils.CopyOnWriteArrayList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -193,9 +192,7 @@ public void sendMessages(List entries) { TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -consumer.sendMessages(entries).getTotalSentMessages()); } else { entries.forEach(entry -> { - MessageMetadata msgMetatada = Consumer.peekMessageMetadata(entry.getDataBuffer(), subscription, -1); - int totalMsgs = msgMetatada.getNumMessagesInBatch(); - msgMetatada.recycle(); + int totalMsgs = Consumer.getNumberOfMessagesInBatch(entry.getDataBuffer(), subscription, -1); if (totalMsgs > 0) { msgDrop.recordEvent(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java index 4cdf91d8cadf3..385e1f781f5b8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java @@ -31,7 +31,6 @@ import org.apache.pulsar.broker.service.RedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; import org.apache.pulsar.broker.service.Subscription; -import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.Policies; @@ -61,9 +60,7 @@ public void sendMessages(List entries) { currentConsumer.sendMessages(entries); } else { entries.forEach(entry -> { - MessageMetadata msgMetatada = Consumer.peekMessageMetadata(entry.getDataBuffer(), subscription, -1); - int totalMsgs = msgMetatada.getNumMessagesInBatch(); - msgMetatada.recycle(); + int totalMsgs = Consumer.getNumberOfMessagesInBatch(entry.getDataBuffer(), subscription, -1); if (totalMsgs > 0) { msgDrop.recordEvent(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java new file mode 100644 index 0000000000000..fbfc4d2d8b1aa --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java @@ -0,0 +1,156 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.delayed; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.*; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +import io.netty.util.Timeout; +import io.netty.util.Timer; +import io.netty.util.TimerTask; + +import java.time.Clock; +import java.util.Collections; +import java.util.NavigableMap; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import lombok.Cleanup; + +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.testng.annotations.Test; + +public class InMemoryDeliveryTrackerTest { + + @Test + public void test() throws Exception { + PersistentDispatcherMultipleConsumers dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + + Timer timer = mock(Timer.class); + + AtomicLong clockTime = new AtomicLong(); + Clock clock = mock(Clock.class); + when(clock.millis()).then(x -> clockTime.get()); + + @Cleanup + InMemoryDelayedDeliveryTracker tracker = new InMemoryDelayedDeliveryTracker(dispatcher, timer, 1, clock); + + assertEquals(tracker.hasMessageAvailable(), false); + + assertTrue(tracker.addMessage(2, 2, 20)); + assertTrue(tracker.addMessage(1, 1, 10)); + assertTrue(tracker.addMessage(3, 3, 30)); + assertTrue(tracker.addMessage(5, 5, 50)); + assertTrue(tracker.addMessage(4, 4, 40)); + + assertEquals(tracker.hasMessageAvailable(), false); + assertEquals(tracker.getNumberOfDelayedMessages(), 5); + + assertEquals(tracker.getScheduledMessages(10), Collections.emptySet()); + + // Move time forward + clockTime.set(15); + + // Message is rejected by tracker since it's already ready to send + assertFalse(tracker.addMessage(6, 6, 10)); + + assertEquals(tracker.getNumberOfDelayedMessages(), 5); + assertEquals(tracker.hasMessageAvailable(), true); + Set scheduled = tracker.getScheduledMessages(10); + assertEquals(scheduled.size(), 1); + + // Move time forward + clockTime.set(60); + + assertEquals(tracker.getNumberOfDelayedMessages(), 4); + assertEquals(tracker.hasMessageAvailable(), true); + scheduled = tracker.getScheduledMessages(1); + assertEquals(scheduled.size(), 1); + + assertEquals(tracker.getNumberOfDelayedMessages(), 3); + assertEquals(tracker.hasMessageAvailable(), true); + scheduled = tracker.getScheduledMessages(3); + assertEquals(scheduled.size(), 3); + + assertEquals(tracker.getNumberOfDelayedMessages(), 0); + assertEquals(tracker.hasMessageAvailable(), false); + assertEquals(tracker.getScheduledMessages(10), Collections.emptySet()); + } + + @Test + public void testWithTimer() throws Exception { + PersistentDispatcherMultipleConsumers dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + Timer timer = mock(Timer.class); + + AtomicLong clockTime = new AtomicLong(); + Clock clock = mock(Clock.class); + when(clock.millis()).then(x -> clockTime.get()); + + NavigableMap tasks = new TreeMap<>(); + + when(timer.newTimeout(any(), anyLong(), any())).then(invocation -> { + TimerTask task = invocation.getArgumentAt(0, TimerTask.class); + long timeout = invocation.getArgumentAt(1, Long.class); + TimeUnit unit = invocation.getArgumentAt(2, TimeUnit.class); + long scheduleAt = clockTime.get() + unit.toMillis(timeout); + tasks.put(scheduleAt, task); + + Timeout t = mock(Timeout.class); + when(t.cancel()).then(i -> { + tasks.remove(scheduleAt, task); + return null; + }); + return t; + }); + + @Cleanup + InMemoryDelayedDeliveryTracker tracker = new InMemoryDelayedDeliveryTracker(dispatcher, timer, 1, clock); + + assertTrue(tasks.isEmpty()); + assertTrue(tracker.addMessage(2, 2, 20)); + assertEquals(tasks.size(), 1); + assertEquals(tasks.firstKey().longValue(), 20); + + assertTrue(tracker.addMessage(1, 1, 10)); + assertEquals(tasks.size(), 1); + assertEquals(tasks.firstKey().longValue(), 10); + + assertTrue(tracker.addMessage(3, 3, 30)); + assertEquals(tasks.size(), 1); + assertEquals(tasks.firstKey().longValue(), 10); + + clockTime.set(15); + + TimerTask task = tasks.pollFirstEntry().getValue(); + Timeout cancelledTimeout = mock(Timeout.class); + when(cancelledTimeout.isCancelled()).thenReturn(true); + task.run(cancelledTimeout); + verifyZeroInteractions(dispatcher); + + task.run(mock(Timeout.class)); + verify(dispatcher).readMoreEntries(); + } +} From ab584c12bcfa5023a6ca5fa4d38c729dac8b5e9c Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 29 Apr 2019 17:18:42 -0700 Subject: [PATCH 06/15] More tests for TripleLongPriorityQueue --- .../TripleLongPriorityQueueTest.java | 87 ++++++++++++++++++- 1 file changed, 86 insertions(+), 1 deletion(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueueTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueueTest.java index 9829c424ed05f..ea04a6d372f7a 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueueTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueueTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.common.util.collections; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.fail; import org.testng.annotations.Test; @@ -29,13 +31,14 @@ public void testQueue() { TripleLongPriorityQueue pq = new TripleLongPriorityQueue(); assertEquals(pq.size(), 0); - final int N = 20; + final int N = 1000; for (int i = N; i > 0; i--) { pq.add(i, i * 2, i * 3); } assertEquals(pq.size(), N); + assertFalse(pq.isEmpty()); for (int i = 1; i <= N; i++) { assertEquals(pq.peekN1(), i); @@ -49,4 +52,86 @@ public void testQueue() { pq.close(); } + + @Test + public void testCheckForEmpty() { + TripleLongPriorityQueue pq = new TripleLongPriorityQueue(); + assertEquals(pq.size(), 0); + assertEquals(pq.isEmpty(), true); + + try { + pq.peekN1(); + fail("Should fail"); + } catch (IllegalArgumentException e) { + // Ok + } + + try { + pq.peekN2(); + fail("Should fail"); + } catch (IllegalArgumentException e) { + // Ok + } + + try { + pq.peekN3(); + fail("Should fail"); + } catch (IllegalArgumentException e) { + // Ok + } + + try { + pq.pop(); + fail("Should fail"); + } catch (IllegalArgumentException e) { + // Ok + } + + pq.close(); + } + + @Test + public void testCompareWithSamePrefix() { + TripleLongPriorityQueue pq = new TripleLongPriorityQueue(); + assertEquals(pq.size(), 0); + assertEquals(pq.isEmpty(), true); + + pq.add(10, 20, 30); + pq.add(20, 10, 10); + pq.add(10, 20, 10); + pq.add(10, 30, 10); + pq.add(10, 20, 5); + + assertEquals(pq.size(), 5); + + assertEquals(pq.peekN1(), 10); + assertEquals(pq.peekN2(), 20); + assertEquals(pq.peekN3(), 5); + pq.pop(); + + assertEquals(pq.peekN1(), 10); + assertEquals(pq.peekN2(), 20); + assertEquals(pq.peekN3(), 10); + pq.pop(); + + assertEquals(pq.peekN1(), 10); + assertEquals(pq.peekN2(), 20); + assertEquals(pq.peekN3(), 30); + pq.pop(); + + assertEquals(pq.peekN1(), 10); + assertEquals(pq.peekN2(), 30); + assertEquals(pq.peekN3(), 10); + pq.pop(); + + assertEquals(pq.peekN1(), 20); + assertEquals(pq.peekN2(), 10); + assertEquals(pq.peekN3(), 10); + pq.pop(); + + assertEquals(pq.size(), 0); + assertEquals(pq.isEmpty(), true); + + pq.close(); + } } From ed16288737dbb1da78bb7cfcc1fec693c86cfae8 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 1 May 2019 10:48:22 -0700 Subject: [PATCH 07/15] Removing useless sync block that causes deadlock with consumer close --- .../broker/service/persistent/PersistentSubscription.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 357f88a5be967..52a7d8b0cc4ef 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 @@ -662,7 +662,7 @@ public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List } @Override - public synchronized void addUnAckedMessages(int unAckMessages) { + public void addUnAckedMessages(int unAckMessages) { dispatcher.addUnAckedMessages(unAckMessages); } From 88a787ba1dfa5089a937a40ad5c02562e8a4efa6 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 1 May 2019 11:12:24 -0700 Subject: [PATCH 08/15] Fixed merge conflict --- .../PersistentStickyKeyDispatcherMultipleConsumers.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 eaa3d1d79765a..d635425f93dcb 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 @@ -92,7 +92,7 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { // remove positions first from replay list first : sendMessages recycles entries List subList = new ArrayList<>(entriesWithSameKey.getValue().subList(0, messagesForC)); if (readType == ReadType.Replay) { - subList.forEach(entry -> messagesToReplay.remove(entry.getLedgerId(), entry.getEntryId())); + subList.forEach(entry -> messagesToRedeliver.remove(entry.getLedgerId(), entry.getEntryId())); } final SendMessageInfo sentMsgInfo = consumer.sendMessages(subList); entriesWithSameKey.getValue().removeAll(subList); @@ -123,7 +123,7 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { for (List entryList : groupedEntries.values()) { laterReplay += entryList.size(); entryList.forEach(entry -> { - messagesToReplay.add(entry.getLedgerId(), entry.getEntryId()); + messagesToRedeliver.add(entry.getLedgerId(), entry.getEntryId()); entry.release(); }); } From c4e4fa89faff003e2d58b1be4215eb1d84131f66 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 2 May 2019 18:24:59 -0700 Subject: [PATCH 09/15] Avoid new list when passing entries to consumer --- .../pulsar/broker/service/Consumer.java | 19 ++++++++++++------- ...PersistentDispatcherMultipleConsumers.java | 4 +--- 2 files changed, 13 insertions(+), 10 deletions(-) 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 a09d6acfb59ee..fdcf2fb543b65 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 @@ -221,7 +221,7 @@ public SendMessageInfo sendMessages(final List entries, SendListener list } try { - updatePermitsAndPendingAcks(entries, sentMessages); + updatePermitsAndFilterMessages(entries, sentMessages); } catch (PulsarServerException pe) { log.warn("[{}] [{}] consumer doesn't support batch-message {}", subscription, consumerId, cnx.getRemoteEndpointProtocolVersion()); @@ -238,6 +238,11 @@ public SendMessageInfo sendMessages(final List entries, SendListener list ctx.channel().eventLoop().execute(() -> { for (int i = 0; i < entries.size(); i++) { Entry entry = entries.get(i); + if (entry == null) { + // Skip deleted entry + continue; + } + PositionImpl pos = (PositionImpl) entry.getPosition(); MessageIdData.Builder messageIdBuilder = MessageIdData.newBuilder(); MessageIdData messageId = messageIdBuilder @@ -310,14 +315,14 @@ public static MessageMetadata peekMessageMetadata(ByteBuf metadataAndPayload, Su } } - void updatePermitsAndPendingAcks(final List entries, SendMessageInfo sentMessages) throws PulsarServerException { + private void updatePermitsAndFilterMessages(final List entries, SendMessageInfo sentMessages) throws PulsarServerException { int permitsToReduce = 0; - Iterator iter = entries.iterator(); boolean unsupportedVersion = false; long totalReadableBytes = 0; boolean clientSupportBatchMessages = cnx.isBatchMessageCompatibleVersion(); - while (iter.hasNext()) { - Entry entry = iter.next(); + + for (int i = 0, entriesSize = entries.size(); i < entriesSize; i++) { + Entry entry = entries.get(i); ByteBuf metadataAndPayload = entry.getDataBuffer(); PositionImpl pos = (PositionImpl) entry.getPosition(); @@ -327,7 +332,7 @@ void updatePermitsAndPendingAcks(final List entries, SendMessageInfo sent try { if (msgMetadata == null) { // Message metadata was corrupted - iter.remove(); + entries.set(i, null); entry.release(); subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual, Collections.emptyMap()); @@ -336,7 +341,7 @@ void updatePermitsAndPendingAcks(final List entries, SendMessageInfo sent && subscription.getDispatcher() .trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { // The message is marked for delayed delivery. Ignore for now. - iter.remove(); + entries.set(i, null); entry.release(); continue; } 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 11b4482fea132..36daf4be0e516 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 @@ -24,7 +24,6 @@ import com.google.common.collect.ComparisonChain; import com.google.common.collect.Lists; -import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.Set; @@ -450,8 +449,7 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { }); } - SendMessageInfo sentMsgInfo = c - .sendMessages(new ArrayList<>(entries.subList(start, start + messagesForC))); + SendMessageInfo sentMsgInfo = c.sendMessages(entries.subList(start, start + messagesForC)); long msgSent = sentMsgInfo.getTotalSentMessages(); start += messagesForC; From 1bbb032b490e5802258a2620566f3031360935b0 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 3 May 2019 09:49:13 -0700 Subject: [PATCH 10/15] Fixed test. Since entries are evicted from cache, they might be resent in diff order --- .../broker/service/persistent/DelayedDeliveryTest.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index 51774ae51973f..5b0461fde2da2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -95,9 +95,15 @@ public void testDelayedDelivery() assertEquals(msg.getValue(), "msg-" + i); } + Set receivedMsgs = new TreeSet<>(); for (int i = 0; i < 10; i++) { msg = sharedConsumer.receive(10, TimeUnit.SECONDS); - assertEquals(msg.getValue(), "msg-" + i); + receivedMsgs.add(msg.getValue()); + } + + assertEquals(receivedMsgs.size(), 10); + for (int i = 0; i < 10; i++) { + assertTrue(receivedMsgs.contains("msg-" + i)); } } From 8180c836a117edd9955152d1f77d4fc2c10e0fa4 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 6 May 2019 17:16:14 -0700 Subject: [PATCH 11/15] Fixed context message builder --- .../pulsar/functions/instance/ContextImpl.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java index f0d5ade6cdf76..459c98e9e13e4 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java @@ -529,6 +529,18 @@ public TypedMessageBuilder loadConf(Map config) { return this; } + @Override + public TypedMessageBuilder deliverAfter(long delay, TimeUnit unit) { + underlyingBuilder.deliverAfter(delay, unit); + return this; + } + + @Override + public TypedMessageBuilder deliverAt(long timestamp) { + underlyingBuilder.deliverAt(timestamp); + return this; + } + public void setUnderlyingBuilder(TypedMessageBuilder underlyingBuilder) { this.underlyingBuilder = underlyingBuilder; } From 640b4bd60562639f1c9a87d2c5b190c832a2e9e2 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 20 May 2019 16:41:28 -0700 Subject: [PATCH 12/15] Fixed triggering writePromise when last entry was nullified --- .../pulsar/broker/service/Consumer.java | 20 +++++--- .../persistent/DelayedDeliveryTest.java | 49 +++++++++++++++++++ 2 files changed, 61 insertions(+), 8 deletions(-) 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 fdcf2fb543b65..0a37656f2192b 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 @@ -24,6 +24,7 @@ import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; @@ -235,6 +236,12 @@ public SendMessageInfo sendMessages(final List entries, SendListener list return sentMessages; } + if (sentMessages.totalSentMessages == 0) { + // Everything was filtered out + writePromise.setSuccess(); + return sentMessages; + } + ctx.channel().eventLoop().execute(() -> { for (int i = 0; i < entries.size(); i++) { Entry entry = entries.get(i); @@ -264,19 +271,16 @@ public SendMessageInfo sendMessages(final List entries, SendListener list consumerId, pos.getLedgerId(), pos.getEntryId()); } - // We only want to pass the "real" promise on the last entry written - ChannelPromise promise = ctx.voidPromise(); - if (i == (entries.size() - 1)) { - promise = writePromise; - } - int redeliveryCount = subscription.getDispatcher().getRedeliveryTracker().getRedeliveryCount(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())); - ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload), promise); + int redeliveryCount = subscription.getDispatcher().getRedeliveryTracker() + .getRedeliveryCount(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())); + ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload), ctx.voidPromise()); messageId.recycle(); messageIdBuilder.recycle(); entry.release(); } - ctx.flush(); + // Use an empty write here so that we can just tie the flush with the write promise for last entry + ctx.writeAndFlush(Unpooled.EMPTY_BUFFER, writePromise); }); return sentMessages; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index 5b0461fde2da2..de8c6a61be900 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -158,4 +158,53 @@ public void testInterleavedMessages() assertTrue(delayedMessages.contains("delayed-msg-" + i)); } } + + @Test + public void testEverythingFilteredInMultipleReads() + throws Exception { + String topic = "testEverythingFilteredInMultipleReads-" + System.nanoTime(); + + @Cleanup + Consumer sharedConsumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("shared-sub") + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + + for (int i = 0; i < 10; i++) { + producer.newMessage() + .value("msg-" + i) + .deliverAfter(5, TimeUnit.SECONDS) + .send(); + } + + Thread.sleep(1000); + + // Write a 2nd batch of messages + for (int i = 10; i < 20; i++) { + producer.newMessage() + .value("msg-" + i) + .deliverAfter(5, TimeUnit.SECONDS) + .send(); + } + + Message msg = sharedConsumer.receive(100, TimeUnit.MILLISECONDS); + assertEquals(msg, null); + + Set receivedMsgs = new TreeSet<>(); + for (int i = 0; i < 20; i++) { + msg = sharedConsumer.receive(10, TimeUnit.SECONDS); + receivedMsgs.add(msg.getValue()); + } + + assertEquals(receivedMsgs.size(), 20); + for (int i = 0; i < 10; i++) { + assertTrue(receivedMsgs.contains("msg-" + i)); + } + } } From dcfa102fd01f5524641c355308544a529c04016a Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 21 May 2019 10:03:48 -0700 Subject: [PATCH 13/15] Moved entries filtering from consumer to dispatcher --- .../service/AbstractBaseDispatcher.java | 116 +++++++++++++ .../AbstractDispatcherMultipleConsumers.java | 11 +- ...bstractDispatcherSingleActiveConsumer.java | 5 +- .../pulsar/broker/service/Consumer.java | 158 +++--------------- .../broker/service/SendMessageInfo.java | 47 ++++++ ...PersistentDispatcherMultipleConsumers.java | 14 +- ...sistentDispatcherSingleActiveConsumer.java | 14 +- ...PersistentDispatcherMultipleConsumers.java | 26 ++- ...sistentDispatcherSingleActiveConsumer.java | 70 ++++---- ...tStickyKeyDispatcherMultipleConsumers.java | 35 ++-- .../persistent/PersistentSubscription.java | 8 +- ...sistentDispatcherFailoverConsumerTest.java | 12 +- .../broker/service/PersistentTopicTest.java | 4 +- .../apache/pulsar/common/api/Commands.java | 32 ++++ 14 files changed, 334 insertions(+), 218 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SendMessageInfo.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java new file mode 100644 index 0000000000000..eba01d86c8ce5 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -0,0 +1,116 @@ +/** + * 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 com.google.common.base.Preconditions.checkArgument; + +import io.netty.buffer.ByteBuf; +import io.netty.util.concurrent.FastThreadLocal; + +import java.util.Collections; +import java.util.List; + +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.common.api.Commands; +import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; +import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; + +public abstract class AbstractBaseDispatcher { + + protected final Subscription subscription; + + protected AbstractBaseDispatcher(Subscription subscription) { + this.subscription = subscription; + } + + /** + * Filter messages that are being sent to a consumers. + *

+ * Messages can be filtered out for multiple reasons: + *

    + *
  • Checksum or metadata corrupted + *
  • Message is an internal marker + *
  • Message is not meant to be delivered immediately + *
+ * + * @param entries + * a list of entries as read from storage + * + * @param batchSizes + * an array where the batch size for each entry (the number of messages within an entry) is stored. This + * array needs to be of at least the same size as the entries list + * + * @param sendMessageInfo + * an object where the total size in messages and bytes will be returned back to the caller + * @param subscription + * the subscription object + */ + public void filterEntriesForConsumer(List entries, int[] batchSizes, SendMessageInfo sendMessageInfo) { + checkArgument(batchSizes.length >= entries.size()); + int totalMessages = 0; + long totalBytes = 0; + + for (int i = 0, entriesSize = entries.size(); i < entriesSize; i++) { + Entry entry = entries.get(i); + ByteBuf metadataAndPayload = entry.getDataBuffer(); + PositionImpl pos = (PositionImpl) entry.getPosition(); + + MessageMetadata msgMetadata = Commands.peekMessageMetadata(metadataAndPayload, subscription.toString(), -1); + + try { + if (msgMetadata == null) { + // Message metadata was corrupted + entries.set(i, null); + entry.release(); + subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual, + Collections.emptyMap()); + continue; + } + + int batchSize = msgMetadata.getNumMessagesInBatch(); + totalMessages += batchSize; + totalBytes += metadataAndPayload.readableBytes(); + batchSizes[i] = batchSize; + } finally { + msgMetadata.recycle(); + } + } + + sendMessageInfo.setTotalMessages(totalMessages); + sendMessageInfo.setTotalBytes(totalBytes); + } + + private static final FastThreadLocal threadLocalBatchSizes = new FastThreadLocal() { + protected int[] initialValue() throws Exception { + return new int[100]; + }; + }; + + protected int[] getThreadLocalBatchSizes(int entries) { + int[] a = threadLocalBatchSizes.get(); + if (a.length < entries) { + a = new int[entries]; + threadLocalBatchSizes.set(a); + } + + return a; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java index e50175566522a..d66375227f8b6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java @@ -28,7 +28,7 @@ /** */ -public abstract class AbstractDispatcherMultipleConsumers { +public abstract class AbstractDispatcherMultipleConsumers extends AbstractBaseDispatcher { protected final CopyOnWriteArrayList consumerList = new CopyOnWriteArrayList<>(); protected final ObjectSet consumerSet = new ObjectHashSet<>(); @@ -39,6 +39,11 @@ public abstract class AbstractDispatcherMultipleConsumers { protected static final AtomicIntegerFieldUpdater IS_CLOSED_UPDATER = AtomicIntegerFieldUpdater .newUpdater(AbstractDispatcherMultipleConsumers.class, "isClosed"); private volatile int isClosed = FALSE; + + protected AbstractDispatcherMultipleConsumers(Subscription subscription) { + super(subscription); + } + public boolean isConsumerConnected() { return !consumerList.isEmpty(); } @@ -127,7 +132,7 @@ public Consumer getNextConsumer() { /** * Finds index of first available consumer which has higher priority then given targetPriority - * + * * @param targetPriority * @return -1 if couldn't find any available consumer */ @@ -187,7 +192,7 @@ private int getNextConsumerFromSameOrLowerLevel(int currentRoundRobinIndex) { /** * Finds index of first consumer in list which has same priority as given targetPriority - * + * * @param targetPriority * @return */ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java index 5bc6af7ad03e7..cd1bf05b2faf7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java @@ -33,7 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class AbstractDispatcherSingleActiveConsumer { +public abstract class AbstractDispatcherSingleActiveConsumer extends AbstractBaseDispatcher { protected final String topicName; protected static final AtomicReferenceFieldUpdater ACTIVE_CONSUMER_UPDATER = @@ -53,7 +53,8 @@ public abstract class AbstractDispatcherSingleActiveConsumer { private volatile int isClosed = FALSE; public AbstractDispatcherSingleActiveConsumer(SubType subscriptionType, int partitionIndex, - String topicName) { + String topicName, Subscription subscription) { + super(subscription); this.topicName = topicName; this.consumers = new CopyOnWriteArrayList<>(); this.partitionIndex = partitionIndex; 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 8934ff4b9368c..d75e51a0adb86 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 @@ -24,14 +24,13 @@ import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelFuture; +import io.netty.buffer.Unpooled; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPromise; import java.util.ArrayList; import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -39,18 +38,14 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.stream.Collectors; -import lombok.Data; - import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.util.Rate; import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap; import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap.LongPair; -import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.common.api.Commands; -import org.apache.pulsar.common.api.proto.PulsarApi; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition; @@ -108,10 +103,6 @@ public class Consumer { private final Map metadata; - public interface SendListener { - void sendComplete(ChannelFuture future, SendMessageInfo sendMessageInfo); - } - public Consumer(Subscription subscription, SubType subType, String topicName, long consumerId, int priorityLevel, String consumerName, int maxUnackedMessages, ServerCnx cnx, String appId, @@ -190,25 +181,9 @@ public boolean readCompacted() { * * @return a SendMessageInfo object that contains the detail of what was sent to consumer */ - public SendMessageInfo sendMessages(final List entries) { - // Empty listener - return sendMessages(entries, null); - } - - /** - * Dispatch a list of entries to the consumer.
- * It is also responsible to release entries data and recycle entries object. - * - * @return a SendMessageInfo object that contains the detail of what was sent to consumer - */ - public SendMessageInfo sendMessages(final List entries, SendListener listener) { + public ChannelPromise sendMessages(final List entries, int[] batchSizes, SendMessageInfo sendMessageInfo) { final ChannelHandlerContext ctx = cnx.ctx(); - final SendMessageInfo sentMessages = new SendMessageInfo(); - final ChannelPromise writePromise = listener != null ? ctx.newPromise() : ctx.voidPromise(); - - if (listener != null) { - writePromise.addListener(future -> listener.sendComplete(writePromise, sentMessages)); - } + final ChannelPromise writePromise = ctx.newPromise(); if (entries.isEmpty()) { if (log.isDebugEnabled()) { @@ -216,29 +191,22 @@ public SendMessageInfo sendMessages(final List entries, SendListener list topicName, subscription, consumerId); } writePromise.setSuccess(); - sentMessages.totalSentMessages = 0; - sentMessages.totalSentMessageBytes = 0; - return sentMessages; + return writePromise; } - try { - updatePermitsAndPendingAcks(entries, sentMessages); - } catch (PulsarServerException pe) { - log.warn("[{}] [{}] consumer doesn't support batch-message {}", subscription, consumerId, - cnx.getRemoteEndpointProtocolVersion()); - - subscription.markTopicWithBatchMessagePublished(); - sentMessages.totalSentMessages = 0; - sentMessages.totalSentMessageBytes = 0; - // disconnect consumer: it will update dispatcher's availablePermits and resend pendingAck-messages of this - // consumer to other consumer - disconnect(); - return sentMessages; - } + // reduce permit and increment unackedMsg count with total number of messages in batch-msgs + MESSAGE_PERMITS_UPDATER.addAndGet(this, -sendMessageInfo.getTotalMessages()); + incrementUnackedMessages(sendMessageInfo.getTotalMessages()); + msgOut.recordMultipleEvents(sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes()); ctx.channel().eventLoop().execute(() -> { for (int i = 0; i < entries.size(); i++) { Entry entry = entries.get(i); + if (entry == null) { + // Entry was filtered out + continue; + } + PositionImpl pos = (PositionImpl) entry.getPosition(); MessageIdData.Builder messageIdBuilder = MessageIdData.newBuilder(); MessageIdData messageId = messageIdBuilder @@ -260,22 +228,19 @@ public SendMessageInfo sendMessages(final List entries, SendListener list consumerId, pos.getLedgerId(), pos.getEntryId()); } - // We only want to pass the "real" promise on the last entry written - ChannelPromise promise = ctx.voidPromise(); - if (i == (entries.size() - 1)) { - promise = writePromise; - } - int redeliveryCount = subscription.getDispatcher().getRedeliveryTracker().getRedeliveryCount(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())); - ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload), promise); + int redeliveryCount = subscription.getDispatcher().getRedeliveryTracker() + .getRedeliveryCount(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())); + ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload), ctx.voidPromise()); messageId.recycle(); messageIdBuilder.recycle(); entry.release(); } - ctx.flush(); + // Use an empty write here so that we can just tie the flush with the write promise for last entry + ctx.writeAndFlush(Unpooled.EMPTY_BUFFER, writePromise); }); - return sentMessages; + return writePromise; } private void incrementUnackedMessages(int ackedMessages) { @@ -284,70 +249,6 @@ private void incrementUnackedMessages(int ackedMessages) { } } - public static int getBatchSizeforEntry(ByteBuf metadataAndPayload, Subscription subscription, long consumerId) { - try { - // save the reader index and restore after parsing - metadataAndPayload.markReaderIndex(); - PulsarApi.MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); - metadataAndPayload.resetReaderIndex(); - int batchSize = metadata.getNumMessagesInBatch(); - metadata.recycle(); - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] num messages in batch are {} ", subscription, consumerId, batchSize); - } - return batchSize; - } catch (Throwable t) { - log.error("[{}] [{}] Failed to parse message metadata", subscription, consumerId, t); - } - return -1; - } - - void updatePermitsAndPendingAcks(final List entries, SendMessageInfo sentMessages) throws PulsarServerException { - int permitsToReduce = 0; - Iterator iter = entries.iterator(); - boolean unsupportedVersion = false; - long totalReadableBytes = 0; - boolean clientSupportBatchMessages = cnx.isBatchMessageCompatibleVersion(); - while (iter.hasNext()) { - Entry entry = iter.next(); - ByteBuf metadataAndPayload = entry.getDataBuffer(); - int batchSize = getBatchSizeforEntry(metadataAndPayload, subscription, consumerId); - if (batchSize == -1) { - // this would suggest that the message might have been corrupted - iter.remove(); - PositionImpl pos = (PositionImpl) entry.getPosition(); - entry.release(); - subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual, Collections.emptyMap()); - continue; - } - if (pendingAcks != null) { - pendingAcks.put(entry.getLedgerId(), entry.getEntryId(), batchSize, 0); - } - // check if consumer supports batch message - if (batchSize > 1 && !clientSupportBatchMessages) { - unsupportedVersion = true; - } - totalReadableBytes += metadataAndPayload.readableBytes(); - permitsToReduce += batchSize; - } - // reduce permit and increment unackedMsg count with total number of messages in batch-msgs - int permits = MESSAGE_PERMITS_UPDATER.addAndGet(this, -permitsToReduce); - incrementUnackedMessages(permitsToReduce); - if (unsupportedVersion) { - throw new PulsarServerException("Consumer does not support batch-message"); - } - if (permits < 0) { - if (log.isDebugEnabled()) { - log.debug("[{}-{}] [{}] message permits dropped below 0 - {}", topicName, subscription, consumerId, - permits); - } - } - - msgOut.recordMultipleEvents(permitsToReduce, totalReadableBytes); - sentMessages.totalSentMessages = permitsToReduce; - sentMessages.totalSentMessageBytes = totalReadableBytes; - } - public boolean isWritable() { return cnx.isWritable(); } @@ -680,26 +581,5 @@ private void clearUnAckedMsgs(Consumer consumer) { subscription.addUnAckedMessages(-unaAckedMsgs); } - public static final class SendMessageInfo { - private int totalSentMessages; - private long totalSentMessageBytes; - - public int getTotalSentMessages() { - return totalSentMessages; - } - - public void setTotalSentMessages(int totalSentMessages) { - this.totalSentMessages = totalSentMessages; - } - - public long getTotalSentMessageBytes() { - return totalSentMessageBytes; - } - - public void setTotalSentMessageBytes(long totalSentMessageBytes) { - this.totalSentMessageBytes = totalSentMessageBytes; - } - } - private static final Logger log = LoggerFactory.getLogger(Consumer.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SendMessageInfo.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SendMessageInfo.java new file mode 100644 index 0000000000000..3cc0ca27d7448 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SendMessageInfo.java @@ -0,0 +1,47 @@ +/** + * 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 io.netty.util.concurrent.FastThreadLocal; + +import lombok.Data; + +@Data +public class SendMessageInfo { + private int totalMessages; + private long totalBytes; + + private SendMessageInfo() { + // Private constructor so that all usages are through the thread-local instance + } + + public static SendMessageInfo getThreadLocal() { + SendMessageInfo smi = THREAD_LOCAL.get(); + smi.totalMessages = 0; + smi.totalBytes = 0; + return smi; + } + + private static final FastThreadLocal THREAD_LOCAL = new FastThreadLocal() { + protected SendMessageInfo initialValue() throws Exception { + return new SendMessageInfo(); + }; + }; + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java index a85552f970c80..c44e88c835fda 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java @@ -18,8 +18,6 @@ */ package org.apache.pulsar.broker.service.nonpersistent; -import static org.apache.pulsar.broker.service.Consumer.getBatchSizeforEntry; - import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -33,7 +31,9 @@ import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.RedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; +import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.common.api.Commands; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.utils.CopyOnWriteArrayList; import org.slf4j.Logger; @@ -59,6 +59,7 @@ public class NonPersistentDispatcherMultipleConsumers extends AbstractDispatcher private final RedeliveryTracker redeliveryTracker; public NonPersistentDispatcherMultipleConsumers(NonPersistentTopic topic, Subscription subscription) { + super(subscription); this.topic = topic; this.subscription = subscription; this.name = topic.getName() + " / " + subscription.getName(); @@ -191,10 +192,15 @@ public RedeliveryTracker getRedeliveryTracker() { public void sendMessages(List entries) { Consumer consumer = TOTAL_AVAILABLE_PERMITS_UPDATER.get(this) > 0 ? getNextConsumer() : null; if (consumer != null) { - TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -consumer.sendMessages(entries).getTotalSentMessages()); + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + int[] batchSizes = getThreadLocalBatchSizes(entries.size()); + filterEntriesForConsumer(entries, batchSizes, sendMessageInfo); + consumer.sendMessages(entries, batchSizes, sendMessageInfo); + + TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -sendMessageInfo.getTotalMessages()); } else { entries.forEach(entry -> { - int totalMsgs = getBatchSizeforEntry(entry.getDataBuffer(), subscription, -1); + int totalMsgs = Commands.getNumberOfMessagesInBatch(entry.getDataBuffer(), subscription.toString(), -1); if (totalMsgs > 0) { msgDrop.recordEvent(totalMsgs); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java index 657e36f03f141..070bd189a169b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.service.nonpersistent; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import static org.apache.pulsar.broker.service.Consumer.getBatchSizeforEntry; import java.util.List; @@ -31,7 +30,9 @@ import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.RedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; +import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.common.api.Commands; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.Policies; @@ -43,10 +44,10 @@ public final class NonPersistentDispatcherSingleActiveConsumer extends AbstractD private final Subscription subscription; private final ServiceConfiguration serviceConfig; private final RedeliveryTracker redeliveryTracker; - + public NonPersistentDispatcherSingleActiveConsumer(SubType subscriptionType, int partitionIndex, NonPersistentTopic topic, Subscription subscription) { - super(subscriptionType, partitionIndex, topic.getName()); + super(subscriptionType, partitionIndex, topic.getName(), subscription); this.topic = topic; this.subscription = subscription; this.msgDrop = new Rate(); @@ -58,10 +59,13 @@ public NonPersistentDispatcherSingleActiveConsumer(SubType subscriptionType, int public void sendMessages(List entries) { Consumer currentConsumer = ACTIVE_CONSUMER_UPDATER.get(this); if (currentConsumer != null && currentConsumer.getAvailablePermits() > 0 && currentConsumer.isWritable()) { - currentConsumer.sendMessages(entries); + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + int[] batchSizes = getThreadLocalBatchSizes(entries.size()); + filterEntriesForConsumer(entries, batchSizes, sendMessageInfo); + currentConsumer.sendMessages(entries, batchSizes, sendMessageInfo); } else { entries.forEach(entry -> { - int totalMsgs = getBatchSizeforEntry(entry.getDataBuffer(), subscription, -1); + int totalMsgs = Commands.getNumberOfMessagesInBatch(entry.getDataBuffer(), subscription.toString(), -1); if (totalMsgs > 0) { msgDrop.recordEvent(totalMsgs); } 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 0bb49652ac987..fc835dd8fe34a 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 @@ -21,6 +21,9 @@ import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.Lists; + import java.util.List; import java.util.Optional; import java.util.Set; @@ -42,12 +45,13 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.Consumer; -import org.apache.pulsar.broker.service.Consumer.SendMessageInfo; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.RedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; import org.apache.pulsar.broker.service.InMemoryRedeliveryTracker; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; +import org.apache.pulsar.broker.service.SendMessageInfo; +import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.TopicName; @@ -59,9 +63,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.ComparisonChain; -import com.google.common.collect.Lists; - /** */ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers implements Dispatcher, ReadEntriesCallback { @@ -95,7 +96,8 @@ enum ReadType { Normal, Replay } - public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor) { + public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor, Subscription subscription) { + super(subscription); this.serviceConfig = topic.getBrokerService().pulsar().getConfiguration(); this.cursor = cursor; this.name = topic.getName() + " / " + Codec.decode(cursor.getName()); @@ -440,14 +442,20 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { }); } - SendMessageInfo sentMsgInfo = c.sendMessages(entries.subList(start, start + messagesForC)); + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + List entriesForThisConsumer = entries.subList(start, start + messagesForC); + + int[] batchSizes = getThreadLocalBatchSizes(entriesForThisConsumer.size()); + filterEntriesForConsumer(entriesForThisConsumer, batchSizes, sendMessageInfo); + + c.sendMessages(entriesForThisConsumer, batchSizes, sendMessageInfo); - long msgSent = sentMsgInfo.getTotalSentMessages(); + long msgSent = sendMessageInfo.getTotalMessages(); start += messagesForC; entriesToDispatch -= messagesForC; totalAvailablePermits -= msgSent; - totalMessagesSent += sentMsgInfo.getTotalSentMessages(); - totalBytesSent += sentMsgInfo.getTotalSentMessageBytes(); + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index e1e17a3650dd5..7e066b65ba69b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -42,6 +42,8 @@ import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.RedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; +import org.apache.pulsar.broker.service.SendMessageInfo; +import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; @@ -68,8 +70,8 @@ public final class PersistentDispatcherSingleActiveConsumer extends AbstractDisp private final RedeliveryTracker redeliveryTracker; public PersistentDispatcherSingleActiveConsumer(ManagedCursor cursor, SubType subscriptionType, int partitionIndex, - PersistentTopic topic) { - super(subscriptionType, partitionIndex, topic.getName()); + PersistentTopic topic, Subscription subscription) { + super(subscriptionType, partitionIndex, topic.getName(), subscription); this.topic = topic; this.name = topic.getName() + " / " + (cursor.getName() != null ? Codec.decode(cursor.getName()) : ""/* NonDurableCursor doesn't have name */); @@ -208,38 +210,46 @@ public synchronized void internalReadEntriesComplete(final List entries, readMoreEntries(currentConsumer); } } else { - currentConsumer.sendMessages(entries, (future, sentMsgInfo) -> { - if (future.isSuccess()) { - // acquire message-dispatch permits for already delivered messages - if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { - if (topic.getDispatchRateLimiter().isPresent()) { - topic.getDispatchRateLimiter().get().tryDispatchPermit(sentMsgInfo.getTotalSentMessages(), - sentMsgInfo.getTotalSentMessageBytes()); - } - - if (dispatchRateLimiter.isPresent()) { - dispatchRateLimiter.get().tryDispatchPermit(sentMsgInfo.getTotalSentMessages(), - sentMsgInfo.getTotalSentMessageBytes()); - } - } + int[] batchSizes = getThreadLocalBatchSizes(entries.size()); + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + filterEntriesForConsumer(entries, batchSizes, sendMessageInfo); + + int totalMessages = sendMessageInfo.getTotalMessages(); + long totalBytes = sendMessageInfo.getTotalBytes(); + + currentConsumer.sendMessages(entries, batchSizes, sendMessageInfo) + .addListener(future -> { + if (future.isSuccess()) { + // acquire message-dispatch permits for already delivered messages + if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { + if (topic.getDispatchRateLimiter().isPresent()) { + topic.getDispatchRateLimiter().get().tryDispatchPermit(totalMessages, totalBytes); + } - // Schedule a new read batch operation only after the previous batch has been written to the socket - topic.getBrokerService().getTopicOrderedExecutor().executeOrdered(topicName, SafeRun.safeRun(() -> { - synchronized (PersistentDispatcherSingleActiveConsumer.this) { - Consumer newConsumer = getActiveConsumer(); - if (newConsumer != null && !havePendingRead) { - readMoreEntries(newConsumer); - } else { - if (log.isDebugEnabled()) { - log.debug( - "[{}-{}] Ignoring write future complete. consumerAvailable={} havePendingRead={}", - name, newConsumer, newConsumer != null, havePendingRead); + if (dispatchRateLimiter.isPresent()) { + dispatchRateLimiter.get().tryDispatchPermit(totalMessages, totalBytes); } } + + // Schedule a new read batch operation only after the previous batch has been written to the + // socket + topic.getBrokerService().getTopicOrderedExecutor().executeOrdered(topicName, + SafeRun.safeRun(() -> { + synchronized (PersistentDispatcherSingleActiveConsumer.this) { + Consumer newConsumer = getActiveConsumer(); + if (newConsumer != null && !havePendingRead) { + readMoreEntries(newConsumer); + } else { + if (log.isDebugEnabled()) { + log.debug( + "[{}-{}] Ignoring write future complete. consumerAvailable={} havePendingRead={}", + name, newConsumer, newConsumer != null, havePendingRead); + } + } + } + })); } - })); - } - }); + }); } } 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 eaa3d1d79765a..2ec4622c87514 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 @@ -19,34 +19,36 @@ package org.apache.pulsar.broker.service.persistent; import io.netty.buffer.ByteBuf; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.Consumer; -import org.apache.pulsar.broker.service.Consumer.SendMessageInfo; import org.apache.pulsar.broker.service.HashRangeStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.common.api.Commands; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDispatcherMultipleConsumers { public static final String NONE_KEY = "NONE_KEY"; private final StickyKeyConsumerSelector selector; - PersistentStickyKeyDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor) { - super(topic, cursor); + PersistentStickyKeyDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor, Subscription subscription) { + super(topic, cursor, subscription); //TODO: Consumer selector Pluggable selector = new HashRangeStickyKeyConsumerSelector(); } @@ -94,12 +96,17 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { if (readType == ReadType.Replay) { subList.forEach(entry -> messagesToReplay.remove(entry.getLedgerId(), entry.getEntryId())); } - final SendMessageInfo sentMsgInfo = consumer.sendMessages(subList); + + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + int[] batchSizes = getThreadLocalBatchSizes(subList.size()); + filterEntriesForConsumer(subList, batchSizes, sendMessageInfo); + + consumer.sendMessages(subList, batchSizes, sendMessageInfo); entriesWithSameKey.getValue().removeAll(subList); - final long msgSent = sentMsgInfo.getTotalSentMessages(); - totalAvailablePermits -= msgSent; - totalMessagesSent += sentMsgInfo.getTotalSentMessages(); - totalBytesSent += sentMsgInfo.getTotalSentMessageBytes(); + + totalAvailablePermits -= sendMessageInfo.getTotalMessages(); + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); if (entriesWithSameKey.getValue().size() == 0) { iterator.remove(); 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 a98af617d25b3..282d6e31261c7 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 @@ -140,12 +140,12 @@ public synchronized void addConsumer(Consumer consumer) throws BrokerServiceExce switch (consumer.subType()) { case Exclusive: if (dispatcher == null || dispatcher.getType() != SubType.Exclusive) { - dispatcher = new PersistentDispatcherSingleActiveConsumer(cursor, SubType.Exclusive, 0, topic); + dispatcher = new PersistentDispatcherSingleActiveConsumer(cursor, SubType.Exclusive, 0, topic, this); } break; case Shared: if (dispatcher == null || dispatcher.getType() != SubType.Shared) { - dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor); + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, this); } break; case Failover: @@ -157,12 +157,12 @@ public synchronized void addConsumer(Consumer consumer) throws BrokerServiceExce if (dispatcher == null || dispatcher.getType() != SubType.Failover) { dispatcher = new PersistentDispatcherSingleActiveConsumer(cursor, SubType.Failover, partitionIndex, - topic); + topic, this); } break; case Key_Shared: if (dispatcher == null || dispatcher.getType() != SubType.Key_Shared) { - dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor); + dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this); } break; default: diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java index cb5a2ef307f12..4b9dfdaf96905 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java @@ -267,7 +267,7 @@ public void testConsumerGroupChangesWithOldNewConsumers() throws Exception { int partitionIndex = 0; PersistentDispatcherSingleActiveConsumer pdfc = new PersistentDispatcherSingleActiveConsumer(cursorMock, - SubType.Failover, partitionIndex, topic); + SubType.Failover, partitionIndex, topic, sub); // 1. Verify no consumers connected assertFalse(pdfc.isConsumerConnected()); @@ -306,7 +306,7 @@ public void testAddRemoveConsumer() throws Exception { int partitionIndex = 0; PersistentDispatcherSingleActiveConsumer pdfc = new PersistentDispatcherSingleActiveConsumer(cursorMock, - SubType.Failover, partitionIndex, topic); + SubType.Failover, partitionIndex, topic, sub); // 1. Verify no consumers connected assertFalse(pdfc.isConsumerConnected()); @@ -421,7 +421,7 @@ public void testAddRemoveConsumer() throws Exception { public void testMultipleDispatcherGetNextConsumerWithDifferentPriorityLevel() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock); + PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(0, 2, false, 1); Consumer consumer2 = createConsumer(0, 2, false, 2); Consumer consumer3 = createConsumer(0, 2, false, 3); @@ -465,7 +465,7 @@ public void testMultipleDispatcherGetNextConsumerWithDifferentPriorityLevel() th @Test public void testFewBlockedConsumerSamePriority() throws Exception{ PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock); + PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(0, 2, false, 1); Consumer consumer2 = createConsumer(0, 2, false, 2); Consumer consumer3 = createConsumer(0, 2, false, 3); @@ -492,7 +492,7 @@ public void testFewBlockedConsumerSamePriority() throws Exception{ @Test public void testFewBlockedConsumerDifferentPriority() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock); + PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(0, 2, false, 1); Consumer consumer2 = createConsumer(0, 2, false, 2); Consumer consumer3 = createConsumer(0, 2, false, 3); @@ -546,7 +546,7 @@ public void testFewBlockedConsumerDifferentPriority() throws Exception { @Test public void testFewBlockedConsumerDifferentPriority2() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock); + PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(0, 2, true, 1); Consumer consumer2 = createConsumer(0, 2, true, 2); Consumer consumer3 = createConsumer(0, 2, true, 3); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index 93664ff651eac..b9cef67681c2e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -285,7 +285,7 @@ public void testDispatcherMultiConsumerReadFailed() throws Exception { PersistentTopic topic = spy(new PersistentTopic(successTopicName, ledgerMock, brokerService)); ManagedCursor cursor = mock(ManagedCursor.class); when(cursor.getName()).thenReturn("cursor"); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor); + PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, null); dispatcher.readEntriesFailed(new ManagedLedgerException.InvalidCursorPositionException("failed"), null); verify(topic, atLeast(1)).getBrokerService(); } @@ -296,7 +296,7 @@ public void testDispatcherSingleConsumerReadFailed() throws Exception { ManagedCursor cursor = mock(ManagedCursor.class); when(cursor.getName()).thenReturn("cursor"); PersistentDispatcherSingleActiveConsumer dispatcher = new PersistentDispatcherSingleActiveConsumer(cursor, - SubType.Exclusive, 1, topic); + SubType.Exclusive, 1, topic, null); Consumer consumer = mock(Consumer.class); dispatcher.readEntriesFailed(new ManagedLedgerException.InvalidCursorPositionException("failed"), consumer); verify(topic, atLeast(1)).getBrokerService(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/Commands.java index 2d76410dd86e8..d1d6cf85db41a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/Commands.java @@ -37,6 +37,9 @@ import java.util.Optional; import java.util.stream.Collectors; +import lombok.experimental.UtilityClass; +import lombok.extern.slf4j.Slf4j; + import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.api.proto.PulsarApi; import org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod; @@ -92,6 +95,8 @@ import org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream; import org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString; +@UtilityClass +@Slf4j public class Commands { // default message size for transfer @@ -1214,6 +1219,33 @@ private static ByteBufPair serializeCommandMessageWithSize(BaseCommand cmd, Byte return (ByteBufPair) ByteBufPair.get(headers, metadataAndPayload); } + public static int getNumberOfMessagesInBatch(ByteBuf metadataAndPayload, String subscription, + long consumerId) { + MessageMetadata msgMetadata = peekMessageMetadata(metadataAndPayload, subscription, consumerId); + if (msgMetadata == null) { + return -1; + } else { + int numMessagesInBatch = msgMetadata.getNumMessagesInBatch(); + msgMetadata.recycle(); + return numMessagesInBatch; + } + } + + public static MessageMetadata peekMessageMetadata(ByteBuf metadataAndPayload, String subscription, + long consumerId) { + try { + // save the reader index and restore after parsing + int readerIdx = metadataAndPayload.readerIndex(); + PulsarApi.MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); + metadataAndPayload.readerIndex(readerIdx); + + return metadata; + } catch (Throwable t) { + log.error("[{}] [{}] Failed to parse message metadata", subscription, consumerId, t); + return null; + } + } + public static int getCurrentProtocolVersion() { // Return the last ProtocolVersion enum value return ProtocolVersion.values()[ProtocolVersion.values().length - 1].getNumber(); From e36a44c40efc4a16601ea967eface1c0ed7d08f7 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 22 May 2019 12:02:17 -0700 Subject: [PATCH 14/15] Added Javadocs --- .../delayed/DelayedDeliveryTrackerLoader.java | 1 - .../collections/TripleLongPriorityQueue.java | 38 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java index b74adcf339f55..90b444d2d5d4a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerLoader.java @@ -30,7 +30,6 @@ public class DelayedDeliveryTrackerLoader { public static DelayedDeliveryTrackerFactory loadDelayedDeliveryTrackerFactory(ServiceConfiguration conf) throws IOException { - // try { Class factoryClass; try { factoryClass = Class.forName(conf.getDelayedDeliveryTrackerFactoryClassName()); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueue.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueue.java index af2673fbb5ffb..9d4b88e842c6a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueue.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/TripleLongPriorityQueue.java @@ -43,21 +43,38 @@ public class TripleLongPriorityQueue implements AutoCloseable { private int capacity; private int size; + /** + * Create a new priority queue with default initial capacity + */ public TripleLongPriorityQueue() { this(DEFAULT_INITIAL_CAPACITY); } + /** + * Create a new priority queue with a given initial capacity + * @param initialCapacity + */ public TripleLongPriorityQueue(int initialCapacity) { capacity = initialCapacity; buffer = PooledByteBufAllocator.DEFAULT.directBuffer(initialCapacity * ITEMS_COUNT * SIZE_OF_LONG); size = 0; } + /** + * Close the priority queue and free the memory associated + */ @Override public void close() { buffer.release(); } + /** + * Add a tuple of 3 long items to the priority queue + * + * @param n1 + * @param n2 + * @param n3 + */ public void add(long n1, long n2, long n3) { if (size == capacity) { increaseCapacity(); @@ -68,16 +85,31 @@ public void add(long n1, long n2, long n3) { ++size; } + /** + * Read the 1st long item in the top tuple in the priority queue. + *

+ * The tuple will not be extracted + */ public long peekN1() { checkArgument(size != 0); return buffer.getLong(0); } + /** + * Read the 2nd long item in the top tuple in the priority queue. + *

+ * The tuple will not be extracted + */ public long peekN2() { checkArgument(size != 0); return buffer.getLong(0 + 1 * SIZE_OF_LONG); } + /** + * Read the 3rd long item in the top tuple in the priority queue. + *

+ * The tuple will not be extracted + */ public long peekN3() { checkArgument(size != 0); return buffer.getLong(0 + 2 * SIZE_OF_LONG); @@ -93,10 +125,16 @@ public void pop() { siftDown(0); } + /** + * Returns whether the priority queue is empty + */ public boolean isEmpty() { return size == 0; } + /** + * Returns the number of tuples in the priority queue + */ public int size() { return size; } From b10712aa14e19468ac89aa482f457441c9999207 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 28 May 2019 14:24:16 -0700 Subject: [PATCH 15/15] Reduced synchronized scope to minimum --- .../PersistentDispatcherMultipleConsumers.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) 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 918d66348aed8..fad8343bd9084 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 @@ -352,11 +352,17 @@ public synchronized boolean canUnsubscribe(Consumer consumer) { } @Override - public synchronized CompletableFuture close() { + public CompletableFuture close() { IS_CLOSED_UPDATER.set(this, TRUE); + + Optional delayedDeliveryTracker; + synchronized (this) { + delayedDeliveryTracker = this.delayedDeliveryTracker; + this.delayedDeliveryTracker = Optional.empty(); + } + if (delayedDeliveryTracker.isPresent()) { delayedDeliveryTracker.get().close(); - delayedDeliveryTracker = Optional.empty(); } return disconnectAllConsumers(); }