-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[feat][broker] Implement topic-level delayed delivery tracking with in-memory manager #24927
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
Denovo1998
wants to merge
16
commits into
apache:master
Choose a base branch
from
Denovo1998:topic_delayed_message
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
16 commits
Select commit
Hold shift + click to select a range
fc876b7
[feat][broker] Implement topic-level delayed delivery tracking with i…
Denovo1998 dee4363
feat[broker] Enhance InMemoryTopicDelayedDeliveryTrackerManager with …
Denovo1998 e455512
[feat][broker] Improve InMemoryTopicDelayedDeliveryTrackerManager wit…
Denovo1998 60f6bcd
feat[broker] Add pruning mechanism and improve timestamp handling in …
Denovo1998 2c30c94
[feat][broker] Refactor delayedMessageMap to use Long2ObjectRBTreeMap…
Denovo1998 6fd6ead
[feat][broker] Refactor mark-delete handling in InMemoryTopicDelayedD…
Denovo1998 6784c43
feat[broker] Simplify InMemoryTopicDelayedDeliveryTrackerManager by r…
Denovo1998 701ffbf
feat[broker] Add testing-friendly constructor and accessors to InMemo…
Denovo1998 c74a879
[feat][broker] Refactor timestamp handling and bucket logic in InMemo…
Denovo1998 e576c65
[feat][broker] Remove deprecated methods and simplify test annotation…
Denovo1998 938e4f6
[feat][broker] Replace highestDeliveryTimeTracked with AtomicLong for…
Denovo1998 e1f2c89
[feat][broker] Introduce InMemoryTopicDelayedDeliveryTrackerFactory f…
Denovo1998 32f2170
[feat][broker] Add configuration options for in-memory topic-level de…
Denovo1998 14fc81b
[feat][broker] Replace manual wait loops with Awaitility for pruning …
Denovo1998 23fbfb6
[feat][broker] Update documentation for InMemoryTopicDelayedDeliveryT…
Denovo1998 4aada39
feat[broker] Centralize mark-delete propagation to topic-level delaye…
Denovo1998 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
119 changes: 119 additions & 0 deletions
119
...r/src/main/java/org/apache/pulsar/broker/delayed/InMemoryTopicDelayedDeliveryTracker.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,119 @@ | ||
| /* | ||
| * 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 java.util.NavigableSet; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import lombok.extern.slf4j.Slf4j; | ||
| import org.apache.bookkeeper.mledger.Position; | ||
|
|
||
| /** | ||
| * Subscription-scoped tracker implementing {@link DelayedDeliveryTracker} by delegating all | ||
| * operations to the topic-level {@link InMemoryTopicDelayedDeliveryTrackerManager}. | ||
| */ | ||
| @Slf4j | ||
| public class InMemoryTopicDelayedDeliveryTracker implements DelayedDeliveryTracker { | ||
|
|
||
| private final InMemoryTopicDelayedDeliveryTrackerManager manager; | ||
| private final InMemoryTopicDelayedDeliveryTrackerManager.SubContext subContext; | ||
| private volatile boolean closed = false; | ||
|
|
||
| public InMemoryTopicDelayedDeliveryTracker(InMemoryTopicDelayedDeliveryTrackerManager manager, | ||
| InMemoryTopicDelayedDeliveryTrackerManager.SubContext subContext) { | ||
| this.manager = manager; | ||
| this.subContext = subContext; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean addMessage(long ledgerId, long entryId, long deliveryAt) { | ||
| checkClosed(); | ||
| return manager.addMessageForSub(subContext, ledgerId, entryId, deliveryAt); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean hasMessageAvailable() { | ||
| checkClosed(); | ||
| return manager.hasMessageAvailableForSub(subContext); | ||
| } | ||
|
|
||
| @Override | ||
| public long getNumberOfDelayedMessages() { | ||
| checkClosed(); | ||
| // Return an estimate of visible delayed messages for this subscription | ||
| // For now, return the total count - could be enhanced to count only visible messages | ||
Denovo1998 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return manager.topicDelayedMessages(); | ||
| } | ||
|
|
||
| @Override | ||
| public long getBufferMemoryUsage() { | ||
| checkClosed(); | ||
| // Return the topic-level memory usage (shared by all subscriptions) | ||
| return manager.topicBufferMemoryBytes(); | ||
| } | ||
|
|
||
| @Override | ||
| public NavigableSet<Position> getScheduledMessages(int maxMessages) { | ||
| checkClosed(); | ||
| return manager.getScheduledMessagesForSub(subContext, maxMessages); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean shouldPauseAllDeliveries() { | ||
| checkClosed(); | ||
| return manager.shouldPauseAllDeliveriesForSub(subContext); | ||
| } | ||
|
|
||
| @Override | ||
| public void resetTickTime(long tickTime) { | ||
| checkClosed(); | ||
| manager.onTickTimeUpdated(tickTime); | ||
| } | ||
|
|
||
| @Override | ||
| public CompletableFuture<Void> clear() { | ||
| checkClosed(); | ||
| // For topic-level manager, clear is a no-op for individual subscriptions | ||
| manager.clearForSub(); | ||
| return CompletableFuture.completedFuture(null); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| if (closed) { | ||
| return; | ||
| } | ||
| closed = true; | ||
| manager.unregister(subContext.getDispatcher()); | ||
| } | ||
|
|
||
| /** | ||
| * Update the mark delete position for this subscription. | ||
| * This is called by the dispatcher when messages are acknowledged. | ||
| */ | ||
| public void updateMarkDeletePosition(Position position) { | ||
| checkClosed(); | ||
| manager.updateMarkDeletePosition(subContext, position); | ||
| } | ||
|
|
||
| private void checkClosed() { | ||
| if (closed) { | ||
| throw new IllegalStateException("DelayedDeliveryTracker is already closed"); | ||
| } | ||
| } | ||
| } | ||
156 changes: 156 additions & 0 deletions
156
...ain/java/org/apache/pulsar/broker/delayed/InMemoryTopicDelayedDeliveryTrackerFactory.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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 com.google.common.annotations.VisibleForTesting; | ||
| import io.netty.util.HashedWheelTimer; | ||
| import io.netty.util.Timer; | ||
| import io.netty.util.concurrent.DefaultThreadFactory; | ||
| import java.time.Clock; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ConcurrentMap; | ||
| import java.util.concurrent.TimeUnit; | ||
| import org.apache.pulsar.broker.PulsarService; | ||
| import org.apache.pulsar.broker.ServiceConfiguration; | ||
| import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| public class InMemoryTopicDelayedDeliveryTrackerFactory implements DelayedDeliveryTrackerFactory { | ||
|
|
||
| private static final Logger log = LoggerFactory.getLogger(InMemoryTopicDelayedDeliveryTrackerFactory.class); | ||
|
|
||
| private Timer timer; | ||
|
|
||
| private long tickTimeMillis; | ||
|
|
||
| private boolean isDelayedDeliveryDeliverAtTimeStrict; | ||
|
|
||
| private long fixedDelayDetectionLookahead; | ||
|
|
||
| // New tuning knobs | ||
| private long pruneMinIntervalMillis; | ||
| private double pruneEligibleRatio; | ||
| private long topicManagerIdleMillis; | ||
|
|
||
| // Cache of topic-level managers: topic name -> manager instance | ||
| private final ConcurrentMap<String, TopicDelayedDeliveryTrackerManager> topicManagers = new ConcurrentHashMap<>(); | ||
|
|
||
| @VisibleForTesting | ||
| InMemoryTopicDelayedDeliveryTrackerFactory(Timer timer, long tickTimeMillis, | ||
| boolean isDelayedDeliveryDeliverAtTimeStrict, | ||
| long fixedDelayDetectionLookahead) { | ||
| this.timer = timer; | ||
| this.tickTimeMillis = tickTimeMillis; | ||
| this.isDelayedDeliveryDeliverAtTimeStrict = isDelayedDeliveryDeliverAtTimeStrict; | ||
| this.fixedDelayDetectionLookahead = fixedDelayDetectionLookahead; | ||
| this.pruneMinIntervalMillis = 0; | ||
| this.pruneEligibleRatio = 0.5; | ||
| this.topicManagerIdleMillis = 0; | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| int getCachedManagersSize() { | ||
| return topicManagers.size(); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| boolean hasManagerForTopic(String topicName) { | ||
| return topicManagers.containsKey(topicName); | ||
| } | ||
|
|
||
| @Override | ||
| public void initialize(PulsarService pulsarService) { | ||
| ServiceConfiguration config = pulsarService.getConfig(); | ||
| this.timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-delayed-delivery"), | ||
| config.getDelayedDeliveryTickTimeMillis(), TimeUnit.MILLISECONDS); | ||
| this.tickTimeMillis = config.getDelayedDeliveryTickTimeMillis(); | ||
| this.isDelayedDeliveryDeliverAtTimeStrict = config.isDelayedDeliveryDeliverAtTimeStrict(); | ||
| this.fixedDelayDetectionLookahead = config.getDelayedDeliveryFixedDelayDetectionLookahead(); | ||
| this.pruneMinIntervalMillis = config.getDelayedDeliveryPruneMinIntervalMillis(); | ||
| this.pruneEligibleRatio = config.getDelayedDeliveryPruneEligibleRatio(); | ||
| this.topicManagerIdleMillis = config.getDelayedDeliveryTopicManagerIdleMillis(); | ||
| } | ||
|
|
||
| @Override | ||
| public DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher) { | ||
| String topicName = dispatcher.getTopic().getName(); | ||
| String subscriptionName = dispatcher.getSubscription().getName(); | ||
| DelayedDeliveryTracker tracker = DelayedDeliveryTracker.DISABLE; | ||
| try { | ||
| tracker = newTracker0(dispatcher); | ||
| } catch (Exception e) { | ||
| // it should never go here | ||
| log.warn("Failed to create InMemoryTopicDelayedDeliveryTracker, topic {}, subscription {}", | ||
| topicName, subscriptionName, e); | ||
| } | ||
| return tracker; | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| DelayedDeliveryTracker newTracker0(AbstractPersistentDispatcherMultipleConsumers dispatcher) { | ||
| String topicName = dispatcher.getTopic().getName(); | ||
|
|
||
| // Get or create topic-level manager for this topic with onEmpty callback to remove from cache | ||
| final TopicDelayedDeliveryTrackerManager[] holder = new TopicDelayedDeliveryTrackerManager[1]; | ||
| TopicDelayedDeliveryTrackerManager manager = topicManagers.computeIfAbsent(topicName, k -> { | ||
| InMemoryTopicDelayedDeliveryTrackerManager m = new InMemoryTopicDelayedDeliveryTrackerManager( | ||
| timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict, | ||
| fixedDelayDetectionLookahead, pruneMinIntervalMillis, pruneEligibleRatio, () -> { | ||
| if (topicManagerIdleMillis <= 0) { | ||
| topicManagers.remove(topicName, holder[0]); | ||
| } else { | ||
| timer.newTimeout(__ -> { | ||
| TopicDelayedDeliveryTrackerManager tm = holder[0]; | ||
| if (tm instanceof InMemoryTopicDelayedDeliveryTrackerManager) { | ||
| if (!((InMemoryTopicDelayedDeliveryTrackerManager) tm).hasActiveSubscriptions()) { | ||
| topicManagers.remove(topicName, tm); | ||
| } | ||
| } else { | ||
| // If the manager has been replaced or removed, ensure entry is cleaned up | ||
| topicManagers.remove(topicName, tm); | ||
| } | ||
| }, topicManagerIdleMillis, TimeUnit.MILLISECONDS); | ||
| } | ||
| }); | ||
| holder[0] = m; | ||
| return m; | ||
| }); | ||
|
|
||
| // Create a per-subscription tracker from the topic-level manager | ||
| return manager.createOrGetTracker(dispatcher); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| // Close all topic-level managers | ||
| for (TopicDelayedDeliveryTrackerManager manager : topicManagers.values()) { | ||
| try { | ||
| manager.close(); | ||
| } catch (Exception e) { | ||
| log.warn("Failed to close topic-level delayed delivery manager", e); | ||
| } | ||
| } | ||
| topicManagers.clear(); | ||
|
|
||
| if (timer != null) { | ||
| timer.stop(); | ||
| } | ||
| } | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.