merlimat commented on a change in pull request #4062: Delayed message delivery implementation URL: https://github.com/apache/pulsar/pull/4062#discussion_r286260160
########## File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java ########## @@ -0,0 +1,193 @@ +/** + * 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.Timeout; +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; + +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 InMemoryDelayedDeliveryTracker implements DelayedDeliveryTracker, 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; + + 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 = clock.millis(); + 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 + */ + @Override + public boolean hasMessageAvailable() { + return !priorityQueue.isEmpty() && priorityQueue.peekN1() <= clock.millis(); + } + + /** + * Get a set of position of messages that have already reached + */ + @Override + public Set<PositionImpl> getScheduledMessages(int maxMessages) { + int n = maxMessages; + Set<PositionImpl> positions = new TreeSet<>(); + 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 + 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(); + --n; + } + + if (log.isDebugEnabled()) { + log.debug("[{}] Get scheduled messags - found {}", dispatcher.getName(), positions.size()); + } + updateTimer(); Review comment: We took items out of the queue, we need to adjust the timer for the next scheduled message ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
