From cccc564ab0ac557001cc9008a155f571a02ad8e6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Feb 2025 14:32:24 +0200 Subject: [PATCH 01/63] [fix][broker] Fix rate limiting causing connections to time out --- .../pulsar/broker/qos/AsyncTokenBucket.java | 19 +- .../qos/DefaultMonotonicSnapshotClock.java | 166 +++++++++++++++--- .../service/PublishRateLimiterImpl.java | 11 +- .../persistent/DispatchRateLimiter.java | 12 +- .../persistent/SubscribeRateLimiter.java | 4 +- .../broker/qos/AsyncTokenBucketTest.java | 97 ++++++++++ .../DefaultMonotonicSnapshotClockTest.java | 93 ++++++++++ 7 files changed, 366 insertions(+), 36 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index ac9a1f03e592b..36edb1c5d8a9e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -159,16 +159,15 @@ private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolea if (shouldUpdateTokensImmediately(currentNanos, forceUpdateTokens)) { // calculate the number of new tokens since the last update long newTokens = calculateNewTokensSinceLastUpdate(currentNanos); - // calculate the total amount of tokens to consume in this update // flush the pendingConsumedTokens by calling "sumThenReset" - long totalConsumedTokens = consumeTokens + pendingConsumedTokens.sumThenReset(); + long currentPendingConsumedTokens = pendingConsumedTokens.sumThenReset(); // update the tokens and return the current token value return TOKENS_UPDATER.updateAndGet(this, - currentTokens -> - // after adding new tokens, limit the tokens to the capacity - Math.min(currentTokens + newTokens, getCapacity()) - // subtract the consumed tokens - - totalConsumedTokens); + // after adding new tokens subtract the pending consumed tokens and + // limit the tokens to the capacity of the bucket + currentTokens -> Math.min(currentTokens + newTokens - currentPendingConsumedTokens, getCapacity()) + // subtract the consumed tokens + - consumeTokens); } else { // eventual consistent fast path, tokens are not updated immediately @@ -211,8 +210,10 @@ private boolean shouldUpdateTokensImmediately(long currentNanos, boolean forceUp */ private long calculateNewTokensSinceLastUpdate(long currentNanos) { long newTokens; - long previousLastNanos = LAST_NANOS_UPDATER.getAndSet(this, currentNanos); - if (previousLastNanos == 0) { + // update lastNanos if currentNanos is greater than the current lastNanos + long previousLastNanos = LAST_NANOS_UPDATER.getAndUpdate(this, + currentLastNanos -> Math.max(currentNanos, currentLastNanos)); + if (previousLastNanos == 0 || previousLastNanos >= currentNanos) { newTokens = 0; } else { long durationNanos = currentNanos - previousLastNanos + REMAINDER_NANOS_UPDATER.getAndSet(this, 0); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index df3843921ed55..a9d8581ae47c7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -29,13 +29,17 @@ * * Starts a daemon thread that updates the snapshot value periodically with a configured interval. The close method * should be called to stop the thread. + * A single thread is used to update the monotonic clock value so that the snapshot value is always increasing, + * even if the clock source is not strictly monotonic across all CPUs. This might be the case in some virtualized + * environments. */ public class DefaultMonotonicSnapshotClock implements MonotonicSnapshotClock, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(DefaultMonotonicSnapshotClock.class); private final long sleepMillis; private final int sleepNanos; private final LongSupplier clockSource; - private final Thread thread; + private final TickUpdaterThread tickUpdaterThread; + private final long snapshotIntervalNanos; private volatile long snapshotTickNanos; public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier clockSource) { @@ -45,45 +49,165 @@ public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier cl this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); this.clockSource = clockSource; - updateSnapshotTickNanos(); - thread = new Thread(this::snapshotLoop, getClass().getSimpleName() + "-update-loop"); - thread.setDaemon(true); - thread.start(); + this.snapshotIntervalNanos = snapshotIntervalNanos; + tickUpdaterThread = new TickUpdaterThread(); + tickUpdaterThread.start(); } /** {@inheritDoc} */ @Override public long getTickNanos(boolean requestSnapshot) { if (requestSnapshot) { - updateSnapshotTickNanos(); + tickUpdaterThread.requestUpdate(); } return snapshotTickNanos; } - private void updateSnapshotTickNanos() { - snapshotTickNanos = clockSource.getAsLong(); - } + /** + * A thread that updates snapshotTickNanos value periodically with a configured interval. + * The thread is started when the DefaultMonotonicSnapshotClock is created and runs until the close method is + * called. + * A single thread is used to read the clock source value since on some hardware of virtualized platforms, + * System.nanoTime() isn't strictly monotonic across all CPUs. Reading by a single thread will improve the + * stability of the read value since a single thread is scheduled on a single CPU. If the thread is migrated + * to another CPU, the clock source value might leap backward or forward, but logic in this class will handle it. + */ + private class TickUpdaterThread extends Thread { + private final Object tickUpdateDelayMonitor = new Object(); + private final Object tickUpdatedMonitor = new Object(); + private final long maxDelta; + private long referenceClockSourceValue; + private long baseSnapshotTickNanos; + private long previousSnapshotTickNanos; + private volatile boolean running = false; + private boolean tickUpdateDelayMonitorNotified = false; + + TickUpdaterThread() { + super(DefaultMonotonicSnapshotClock.class.getSimpleName() + "-update-loop"); + // set as daemon thread so that it doesn't prevent the JVM from exiting + setDaemon(true); + // set the highest priority + setPriority(MAX_PRIORITY); + this.maxDelta = 2 * snapshotIntervalNanos; + } + + @Override + public void run() { + try { + running = true; + // initially update the snapshot value and notify all threads that are waiting for the tick value + // the start method waits until the tick value has been updated + updateSnapshotTickNanos(false); + notifyAllTickUpdated(); + while (!isInterrupted()) { + try { + boolean snapshotRequested; + // sleep for the configured interval on a monitor that can be notified to stop the sleep + // and update the tick value immediately. This is used in requestUpdate method. + synchronized (tickUpdateDelayMonitor) { + tickUpdateDelayMonitorNotified = false; + tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); + snapshotRequested = tickUpdateDelayMonitorNotified; + } + updateSnapshotTickNanos(snapshotRequested); + notifyAllTickUpdated(); + } catch (InterruptedException e) { + interrupt(); + break; + } + } + } catch (Throwable t) { + // report unexpected error since this would be a fatal error when the clock doesn't progress anymore + // this is very unlikely to happen, but it's better to log it in any case + LOG.error("Unexpected fatal error that stopped the clock.", t); + } finally { + LOG.info("DefaultMonotonicSnapshotClock's TickUpdaterThread stopped. {},tid={}", this, getId()); + running = false; + } + } + + private void updateSnapshotTickNanos(boolean snapshotRequested) { + long clockValue = clockSource.getAsLong(); + + // Initialization + if (referenceClockSourceValue == 0) { + referenceClockSourceValue = clockValue; + baseSnapshotTickNanos = clockValue; + snapshotTickNanos = clockValue; + previousSnapshotTickNanos = clockValue; + return; + } + + // calculate the duration since the reference clock source value + // so that the snapshot value is always increasing and tolerates it when the clock source is not strictly + // monotonic across all CPUs and leaps backward or forward + long durationSinceReference = clockValue - referenceClockSourceValue; + long newSnapshotTickNanos = baseSnapshotTickNanos + durationSinceReference; + + // reset the reference clock source value if the clock source value leaps backward or forward + if (newSnapshotTickNanos < previousSnapshotTickNanos - maxDelta + || newSnapshotTickNanos > previousSnapshotTickNanos + maxDelta) { + referenceClockSourceValue = clockValue; + baseSnapshotTickNanos = previousSnapshotTickNanos; + if (!snapshotRequested) { + // if the snapshot value is not requested, increment by the snapshot interval + baseSnapshotTickNanos += snapshotIntervalNanos; + } + newSnapshotTickNanos = baseSnapshotTickNanos; + } + + // update snapshotTickNanos value if the new value is greater than the previous value + if (newSnapshotTickNanos > previousSnapshotTickNanos) { + snapshotTickNanos = newSnapshotTickNanos; + // store into a field so that we don't need to do a volatile read to find out the previous value + previousSnapshotTickNanos = newSnapshotTickNanos; + } + } + + private void notifyAllTickUpdated() { + synchronized (tickUpdatedMonitor) { + // notify all threads that are waiting for the tick value to be updated + tickUpdatedMonitor.notifyAll(); + } + } + + public void requestUpdate() { + if (!running) { + // thread has stopped running, fallback to update the value directly without any optimizations + snapshotTickNanos = clockSource.getAsLong(); + return; + } + synchronized (tickUpdatedMonitor) { + // notify the thread to stop waiting and update the tick value + synchronized (tickUpdateDelayMonitor) { + tickUpdateDelayMonitorNotified = true; + tickUpdateDelayMonitor.notify(); + } + // wait until the tick value has been updated + try { + tickUpdatedMonitor.wait(); + } catch (InterruptedException e) { + currentThread().interrupt(); + } + } + } - private void snapshotLoop() { - try { - while (!Thread.currentThread().isInterrupted()) { - updateSnapshotTickNanos(); + @Override + public synchronized void start() { + super.start(); + // wait until the thread is started and the tick value has been updated + synchronized (tickUpdatedMonitor) { try { - Thread.sleep(sleepMillis, sleepNanos); + tickUpdatedMonitor.wait(); } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - break; + currentThread().interrupt(); } } - } catch (Throwable t) { - // report unexpected error since this would be a fatal error when the clock doesn't progress anymore - // this is very unlikely to happen, but it's better to log it in any case - LOG.error("Unexpected fatal error that stopped the clock.", t); } } @Override public void close() { - thread.interrupt(); + tickUpdaterThread.interrupt(); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java index 8255d9b6931ff..90c8de5f97a05 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java @@ -20,11 +20,11 @@ package org.apache.pulsar.broker.service; import com.google.common.annotations.VisibleForTesting; -import io.netty.channel.EventLoopGroup; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.broker.qos.MonotonicSnapshotClock; import org.apache.pulsar.common.policies.data.Policies; @@ -32,6 +32,7 @@ import org.jctools.queues.MessagePassingQueue; import org.jctools.queues.MpscUnboundedArrayQueue; +@Slf4j public class PublishRateLimiterImpl implements PublishRateLimiter { private volatile AsyncTokenBucket tokenBucketOnMessage; private volatile AsyncTokenBucket tokenBucketOnByte; @@ -80,7 +81,7 @@ private void scheduleDecrementThrottleCount(Producer producer) { // schedule unthrottling when the throttling count is incremented to 1 // this is to avoid scheduling unthrottling multiple times for concurrent producers if (throttledProducersCount.incrementAndGet() == 1) { - EventLoopGroup executor = producer.getCnx().getBrokerService().executor(); + ScheduledExecutorService executor = producer.getCnx().getBrokerService().executor().next(); scheduleUnthrottling(executor, calculateThrottlingDurationNanos()); } } @@ -134,7 +135,11 @@ private void unthrottleQueuedProducers(ScheduledExecutorService executor) { // unthrottle as many producers as possible while there are token available while ((throttlingDuration = calculateThrottlingDurationNanos()) == 0L && (producer = unthrottlingQueue.poll()) != null) { - producer.decrementThrottleCount(); + try { + producer.decrementThrottleCount(); + } catch (Exception e) { + log.error("Failed to unthrottle producer {}", producer, e); + } throttledProducersCount.decrementAndGet(); } // if there are still producers to be unthrottled, schedule unthrottling again diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java index b29cbcd660db1..0f4323e5fef3d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.qos.AsyncTokenBucket; +import org.apache.pulsar.broker.qos.MonotonicSnapshotClock; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -216,18 +217,22 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { long msgRate = dispatchRate.getDispatchThrottlingRateInMsg(); long byteRate = dispatchRate.getDispatchThrottlingRateInByte(); long ratePeriodNanos = TimeUnit.SECONDS.toNanos(Math.max(dispatchRate.getRatePeriodInSecond(), 1)); + MonotonicSnapshotClock clock = brokerService.getPulsar().getMonotonicSnapshotClock(); // update msg-rateLimiter if (msgRate > 0) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnMessage = AsyncTokenBucket.builderForDynamicRate() + .clock(clock) .rateFunction(() -> getRelativeDispatchRateInMsg(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) .build(); } else { this.dispatchRateLimiterOnMessage = - AsyncTokenBucket.builder().rate(msgRate).ratePeriodNanos(ratePeriodNanos) + AsyncTokenBucket.builder() + .clock(clock) + .rate(msgRate).ratePeriodNanos(ratePeriodNanos) .build(); } } else { @@ -239,12 +244,15 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnByte = AsyncTokenBucket.builderForDynamicRate() + .clock(clock) .rateFunction(() -> getRelativeDispatchRateInByte(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) .build(); } else { this.dispatchRateLimiterOnByte = - AsyncTokenBucket.builder().rate(byteRate).ratePeriodNanos(ratePeriodNanos) + AsyncTokenBucket.builder() + .clock(clock) + .rate(byteRate).ratePeriodNanos(ratePeriodNanos) .build(); } } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java index b1de10e73b76f..e9951b83d7947 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java @@ -117,7 +117,9 @@ private synchronized void updateSubscribeRate(ConsumerIdentifier consumerIdentif // update subscribe-rateLimiter if (ratePerConsumer > 0) { AsyncTokenBucket tokenBucket = - AsyncTokenBucket.builder().rate(ratePerConsumer).ratePeriodNanos(ratePeriodNanos).build(); + AsyncTokenBucket.builder() + .clock(brokerService.getPulsar().getMonotonicSnapshotClock()) + .rate(ratePerConsumer).ratePeriodNanos(ratePeriodNanos).build(); this.subscribeRateLimiter.put(consumerIdentifier, tokenBucket); } else { // subscribe-rate should be disable and close diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index b446f9e902f2a..89934803cf1ed 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.qos; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -100,4 +101,100 @@ void shouldSupportFractionsAndRetainLeftoverWhenUpdatingTokens() { assertEquals(asyncTokenBucket.getTokens(), 3); } + @Test + void shouldSupportFractionsAndRetainLeftoverWhenUpdatingTokens2() { + asyncTokenBucket = + AsyncTokenBucket.builder().capacity(100).rate(1).initialTokens(0).clock(clockSource).build(); + for (int i = 0; i < 150; i++) { + incrementMillis(1); + assertEquals(asyncTokenBucket.tokens((i + 1) % 31 == 0), 0); + } + incrementMillis(150); + assertEquals(asyncTokenBucket.tokens(true), 0); + incrementMillis(699); + assertEquals(asyncTokenBucket.tokens(true), 0); + incrementMillis(2); + assertEquals(asyncTokenBucket.tokens(true), 1); + incrementMillis(999); + assertEquals(asyncTokenBucket.tokens(true), 2); + } + + @Test + void shouldHandleNegativeBalanceWithEventuallyConsistentTokenUpdates() { + asyncTokenBucket = + AsyncTokenBucket.builder() + // intentionally pick a coarse resolution + .resolutionNanos(TimeUnit.SECONDS.toNanos(51)) + .capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + // assert that the token balance is 0 initially + assertThat(asyncTokenBucket.tokens(true)).isEqualTo(0); + + // consume tokens without exceeding the rate + for (int i = 0; i < 10000; i++) { + asyncTokenBucket.consumeTokens(500); + incrementSeconds(50); + } + + // let 9 seconds pass + incrementSeconds(9); + + // there should be 90 tokens available + assertThat(asyncTokenBucket.tokens(true)).isEqualTo(90); + } + + @Test + void shouldNotExceedTokenBucketSizeWithNegativeTokens() { + asyncTokenBucket = + AsyncTokenBucket.builder() + // intentionally pick a coarse resolution + .resolutionNanos(TimeUnit.SECONDS.toNanos(51)) + .capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + // assert that the token balance is 0 initially + assertThat(asyncTokenBucket.tokens(true)).isEqualTo(0); + + // consume tokens without exceeding the rate + for (int i = 0; i < 100; i++) { + asyncTokenBucket.consumeTokens(600); + incrementSeconds(50); + // let tokens accumulate back to 0 every 10 seconds + if ((i + 1) % 10 == 0) { + incrementSeconds(100); + } + } + + // let 9 seconds pass + incrementSeconds(9); + + // there should be 90 tokens available + assertThat(asyncTokenBucket.tokens(true)).isEqualTo(90); + } + + @Test + void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates() { + clockSource = requestSnapshot -> { + if (requestSnapshot) { + return manualClockSource.get(); + } else { + // let the clock lag behind + return manualClockSource.get() - TimeUnit.SECONDS.toNanos(52); + } + }; + incrementSeconds(1); + asyncTokenBucket = + AsyncTokenBucket.builder().resolutionNanos(TimeUnit.SECONDS.toNanos(51)) + .capacity(100).rate(10).initialTokens(100).clock(clockSource).build(); + assertThat(asyncTokenBucket.tokens(true)).isEqualTo(100); + + // consume tokens without exceeding the rate + for (int i = 0; i < 10000; i++) { + asyncTokenBucket.consumeTokens(500); + incrementSeconds(i == 0 ? 40 : 50); + } + + // let 9 seconds pass + incrementSeconds(9); + + // there should be 90 tokens available + assertThat(asyncTokenBucket.tokens(true)).isEqualTo(90); + } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java new file mode 100644 index 0000000000000..40ab2c44d8c38 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java @@ -0,0 +1,93 @@ +/* + * 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.qos; + +import static org.assertj.core.api.Assertions.assertThat; +import java.time.Duration; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.assertj.core.data.Offset; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +public class DefaultMonotonicSnapshotClockTest { + @DataProvider + private static Object[] booleanValues() { + return new Object[]{ true, false }; + } + + @Test(dataProvider = "booleanValues") + void testClockHandlesTimeLeapsBackwardsOrForward(boolean requestSnapshot) throws InterruptedException { + long snapshotIntervalMillis = 5; + AtomicLong offsetValue = new AtomicLong(0); + @Cleanup + DefaultMonotonicSnapshotClock clock = + new DefaultMonotonicSnapshotClock(Duration.ofMillis(snapshotIntervalMillis).toNanos(), + () -> System.nanoTime() + offsetValue.get()); + + long previousTick = -1; + boolean leapDirection = true; + for (int i = 0; i < 100; i++) { + long tick = clock.getTickNanos(requestSnapshot); + log.info("i = {}, tick = {}", i, tick); + if ((i + 1) % 3 == 0) { + leapDirection = !leapDirection; + log.info("Time leap 5 minutes {}", leapDirection ? "forward" : "backwards"); + // make the clock leap 5 minute forward or backwards + offsetValue.set((leapDirection ? 1L : -1L) * Duration.ofMinutes(5).toNanos()); + Thread.sleep(2 * snapshotIntervalMillis); + } else { + Thread.sleep(snapshotIntervalMillis); + } + try { + var assertion = assertThat(tick) + .describedAs("i = %d, tick = %d, previousTick = %d", i, tick, previousTick); + if (requestSnapshot) { + assertion = assertion.isGreaterThan(previousTick); + } else { + assertion = assertion.isGreaterThanOrEqualTo(previousTick); + } + assertion.isCloseTo(previousTick, + Offset.offset(5 * TimeUnit.MILLISECONDS.toNanos(snapshotIntervalMillis))); + } catch (AssertionError e) { + if (i < 3) { + // ignore the assertion errors in first 3 rounds since classloading of AssertJ makes the timings + // flaky + } else { + throw e; + } + } + previousTick = tick; + } + } + + @Test + void testRequestUpdate() throws InterruptedException { + @Cleanup + DefaultMonotonicSnapshotClock clock = + new DefaultMonotonicSnapshotClock(Duration.ofSeconds(5).toNanos(), System::nanoTime); + long tick1 = clock.getTickNanos(false); + long tick2 = clock.getTickNanos(true); + assertThat(tick2).isGreaterThan(tick1); + } +} \ No newline at end of file From 79c9cce30ab156649d8375e609fc5fa5a37b3c96 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Feb 2025 16:12:47 +0200 Subject: [PATCH 02/63] Fix test --- .../apache/pulsar/broker/service/PublishRateLimiterTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java index 2c44ba7e23004..5c149d4e1e792 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java @@ -26,6 +26,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import io.netty.channel.EventLoop; import io.netty.channel.EventLoopGroup; import java.util.HashMap; import java.util.concurrent.TimeUnit; @@ -73,7 +74,9 @@ public void setup() throws Exception { when(transportCnx.getBrokerService()).thenReturn(brokerService); EventLoopGroup eventLoopGroup = mock(EventLoopGroup.class); when(brokerService.executor()).thenReturn(eventLoopGroup); - doReturn(null).when(eventLoopGroup).schedule(any(Runnable.class), anyLong(), any()); + EventLoop eventLoop = mock(EventLoop.class); + when(eventLoopGroup.next()).thenReturn(eventLoop); + doReturn(null).when(eventLoop).schedule(any(Runnable.class), anyLong(), any()); incrementSeconds(1); } From 9fa43a98b3b09afb1903dee95ead3116ef11410f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Feb 2025 16:46:11 +0200 Subject: [PATCH 03/63] Add test that checks that clock leaping backward or forward would be tolerated --- .../broker/qos/AsyncTokenBucketTest.java | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index 89934803cf1ed..5e982858a4bd9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -21,8 +21,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; +import java.util.Random; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import org.assertj.core.data.Offset; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -197,4 +199,32 @@ void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates // there should be 90 tokens available assertThat(asyncTokenBucket.tokens(true)).isEqualTo(90); } + + @Test + void shouldTolerateInstableClockSourceWhenUpdatingTokens() { + AtomicLong offset = new AtomicLong(0); + long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(100); + DefaultMonotonicSnapshotClock monotonicSnapshotClock = + new DefaultMonotonicSnapshotClock(resolutionNanos, + () -> offset.get() + manualClockSource.get()); + asyncTokenBucket = + AsyncTokenBucket.builder().resolutionNanos(resolutionNanos) + .capacity(100000).rate(1000).initialTokens(500).clock(monotonicSnapshotClock).build(); + Random random = new Random(0); + int randomOffsetCount = 0; + for (int i = 0; i < 100000; i++) { + incrementMillis(1); + if (i % 39 == 0) { + // randomly offset the clock source + // update the tokens consistently before and after offsetting the clock source + asyncTokenBucket.tokens(true); + offset.set((random.nextBoolean() ? -1L : 1L) * random.nextLong(3L, 100L) * resolutionNanos); + asyncTokenBucket.tokens(true); + randomOffsetCount++; + } + asyncTokenBucket.consumeTokens(1); + } + assertThat(asyncTokenBucket.tokens(true)) + .isGreaterThan(500L).isCloseTo(500L, Offset.offset(3L * randomOffsetCount)); + } } \ No newline at end of file From f28af8530baa9b4fc9d32a606fc52fdbda7e8cba Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Feb 2025 18:12:11 +0200 Subject: [PATCH 04/63] Improve DefaultMonotonicSnapshotClock so that requests don't get delayed when there are races --- .../qos/DefaultMonotonicSnapshotClock.java | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index a9d8581ae47c7..c2a4763e9d634 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -79,8 +79,9 @@ private class TickUpdaterThread extends Thread { private long referenceClockSourceValue; private long baseSnapshotTickNanos; private long previousSnapshotTickNanos; - private volatile boolean running = false; - private boolean tickUpdateDelayMonitorNotified = false; + private volatile boolean running; + private boolean tickUpdateDelayMonitorNotified; + private long requestCount; TickUpdaterThread() { super(DefaultMonotonicSnapshotClock.class.getSimpleName() + "-update-loop"); @@ -95,19 +96,21 @@ private class TickUpdaterThread extends Thread { public void run() { try { running = true; - // initially update the snapshot value and notify all threads that are waiting for the tick value - // the start method waits until the tick value has been updated - updateSnapshotTickNanos(false); - notifyAllTickUpdated(); + long updatedForRequestCount = -1; while (!isInterrupted()) { try { - boolean snapshotRequested; + boolean snapshotRequested = false; // sleep for the configured interval on a monitor that can be notified to stop the sleep // and update the tick value immediately. This is used in requestUpdate method. synchronized (tickUpdateDelayMonitor) { tickUpdateDelayMonitorNotified = false; - tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); - snapshotRequested = tickUpdateDelayMonitorNotified; + // only wait if no explicit request has been made since the last update + if (requestCount == updatedForRequestCount) { + // if no request has been made, sleep for the configured interval + tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); + snapshotRequested = tickUpdateDelayMonitorNotified; + } + updatedForRequestCount = requestCount; } updateSnapshotTickNanos(snapshotRequested); notifyAllTickUpdated(); @@ -123,6 +126,7 @@ public void run() { } finally { LOG.info("DefaultMonotonicSnapshotClock's TickUpdaterThread stopped. {},tid={}", this, getId()); running = false; + notifyAllTickUpdated(); } } @@ -181,6 +185,7 @@ public void requestUpdate() { // notify the thread to stop waiting and update the tick value synchronized (tickUpdateDelayMonitor) { tickUpdateDelayMonitorNotified = true; + requestCount++; tickUpdateDelayMonitor.notify(); } // wait until the tick value has been updated From a99fe0c8797995833d6384bf990c619ab5958180 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Feb 2025 18:15:16 +0200 Subject: [PATCH 05/63] Also test with small offsets --- .../java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index 5e982858a4bd9..eae4aea0af347 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -218,7 +218,7 @@ void shouldTolerateInstableClockSourceWhenUpdatingTokens() { // randomly offset the clock source // update the tokens consistently before and after offsetting the clock source asyncTokenBucket.tokens(true); - offset.set((random.nextBoolean() ? -1L : 1L) * random.nextLong(3L, 100L) * resolutionNanos); + offset.set((random.nextBoolean() ? -1L : 1L) * random.nextLong(0L, 100L) * resolutionNanos); asyncTokenBucket.tokens(true); randomOffsetCount++; } From 907271390277e520b48272510202b249e92c3245 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Feb 2025 18:24:34 +0200 Subject: [PATCH 06/63] Improve test case --- .../pulsar/broker/qos/AsyncTokenBucketTest.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index eae4aea0af347..d12f21c718cc5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -207,12 +207,14 @@ void shouldTolerateInstableClockSourceWhenUpdatingTokens() { DefaultMonotonicSnapshotClock monotonicSnapshotClock = new DefaultMonotonicSnapshotClock(resolutionNanos, () -> offset.get() + manualClockSource.get()); + long initialTokens = 500L; asyncTokenBucket = AsyncTokenBucket.builder().resolutionNanos(resolutionNanos) - .capacity(100000).rate(1000).initialTokens(500).clock(monotonicSnapshotClock).build(); + .capacity(100000).rate(1000).initialTokens(initialTokens).clock(monotonicSnapshotClock).build(); Random random = new Random(0); int randomOffsetCount = 0; for (int i = 0; i < 100000; i++) { + // increment the clock by 1ms, since rate is 1000 tokens/s, this should make 1 token available incrementMillis(1); if (i % 39 == 0) { // randomly offset the clock source @@ -222,9 +224,15 @@ void shouldTolerateInstableClockSourceWhenUpdatingTokens() { asyncTokenBucket.tokens(true); randomOffsetCount++; } + // consume 1 token asyncTokenBucket.consumeTokens(1); } assertThat(asyncTokenBucket.tokens(true)) - .isGreaterThan(500L).isCloseTo(500L, Offset.offset(3L * randomOffsetCount)); + // since the rate is 1/ms and the test increments the clock by 1ms and consumes 1 token in each + // iteration, the tokens should be greater than or equal to the initial tokens + .isGreaterThanOrEqualTo(initialTokens) + // tolerate difference in added tokens since when clock leaps forward or backwards, the clock + // is assumed to have moved forward by the resolutionNanos + .isCloseTo(initialTokens, Offset.offset(3L * randomOffsetCount)); } } \ No newline at end of file From f719dbc70e875f42a1b93a5a9df9733e0cc7e0a9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Feb 2025 18:57:21 +0200 Subject: [PATCH 07/63] Use JMH blackhole in test --- .../pulsar/broker/qos/AsyncTokenBucketBenchmark.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java b/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java index 4c069e72ea3ba..23a9bfe8348bd 100644 --- a/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java +++ b/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java @@ -33,6 +33,7 @@ import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Threads; import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; @Fork(3) @BenchmarkMode(Mode.Throughput) @@ -59,23 +60,26 @@ public void teardown() { @Benchmark @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) - public void consumeTokensBenchmark001Threads() { + public void consumeTokensBenchmark001Threads(Blackhole blackhole) { asyncTokenBucket.consumeTokens(1); + blackhole.consume(asyncTokenBucket.getTokens()); } @Threads(10) @Benchmark @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) - public void consumeTokensBenchmark010Threads() { + public void consumeTokensBenchmark010Threads(Blackhole blackhole) { asyncTokenBucket.consumeTokens(1); + blackhole.consume(asyncTokenBucket.getTokens()); } @Threads(100) @Benchmark @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) - public void consumeTokensBenchmark100Threads() { + public void consumeTokensBenchmark100Threads(Blackhole blackhole) { asyncTokenBucket.consumeTokens(1); + blackhole.consume(asyncTokenBucket.getTokens()); } } From c60c4b835959065f0ee4f6e47612ab449ceaf585 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 10:50:26 +0200 Subject: [PATCH 08/63] Improve code coverage --- .../qos/DefaultMonotonicSnapshotClock.java | 3 +- .../DefaultMonotonicSnapshotClockTest.java | 38 +++++++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index c2a4763e9d634..ffc825244cde0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.qos; +import java.util.Objects; import java.util.concurrent.TimeUnit; import java.util.function.LongSupplier; import org.slf4j.Logger; @@ -48,7 +49,7 @@ public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier cl } this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); - this.clockSource = clockSource; + this.clockSource = Objects.requireNonNull(clockSource, "clockSource must not be null"); this.snapshotIntervalNanos = snapshotIntervalNanos; tickUpdaterThread = new TickUpdaterThread(); tickUpdaterThread.start(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java index 40ab2c44d8c38..121522876ba65 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java @@ -20,6 +20,7 @@ package org.apache.pulsar.broker.qos; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -90,4 +91,41 @@ void testRequestUpdate() throws InterruptedException { long tick2 = clock.getTickNanos(true); assertThat(tick2).isGreaterThan(tick1); } + + @Test + void testRequestingSnapshotAfterClosed() throws InterruptedException { + DefaultMonotonicSnapshotClock clock = + new DefaultMonotonicSnapshotClock(Duration.ofSeconds(5).toNanos(), System::nanoTime); + clock.close(); + long tick1 = clock.getTickNanos(true); + Thread.sleep(10); + long tick2 = clock.getTickNanos(true); + assertThat(tick2).isGreaterThan(tick1); + } + + @Test + void testConstructorValidation() { + assertThatThrownBy(() -> new DefaultMonotonicSnapshotClock(0, System::nanoTime)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("snapshotIntervalNanos must be at least 1 millisecond"); + assertThatThrownBy(() -> new DefaultMonotonicSnapshotClock(-1, System::nanoTime)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("snapshotIntervalNanos must be at least 1 millisecond"); + assertThatThrownBy(() -> new DefaultMonotonicSnapshotClock(TimeUnit.MILLISECONDS.toNanos(1), null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("clockSource must not be null"); + } + + @Test + void testFailureHandlingInClockSource() { + @Cleanup + DefaultMonotonicSnapshotClock clock = + new DefaultMonotonicSnapshotClock(Duration.ofSeconds(5).toNanos(), () -> { + throw new RuntimeException("Test clock failure"); + }); + // the exception should be propagated + assertThatThrownBy(() -> clock.getTickNanos(true)) + .isInstanceOf(RuntimeException.class) + .hasMessage("Test clock failure"); + } } \ No newline at end of file From 696f9fb68c772bc1ea434b997108b07a9a10c94d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 11:53:49 +0200 Subject: [PATCH 09/63] Refactor: Split out logic for leap detection and monotonic tick updating --- .../qos/DefaultMonotonicSnapshotClock.java | 145 +++++++++++------- 1 file changed, 90 insertions(+), 55 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index ffc825244cde0..b6f5de97f522d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -21,6 +21,7 @@ import java.util.Objects; import java.util.concurrent.TimeUnit; +import java.util.function.LongConsumer; import java.util.function.LongSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,8 +37,6 @@ */ public class DefaultMonotonicSnapshotClock implements MonotonicSnapshotClock, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(DefaultMonotonicSnapshotClock.class); - private final long sleepMillis; - private final int sleepNanos; private final LongSupplier clockSource; private final TickUpdaterThread tickUpdaterThread; private final long snapshotIntervalNanos; @@ -47,11 +46,9 @@ public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier cl if (snapshotIntervalNanos < TimeUnit.MILLISECONDS.toNanos(1)) { throw new IllegalArgumentException("snapshotIntervalNanos must be at least 1 millisecond"); } - this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); - this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); this.clockSource = Objects.requireNonNull(clockSource, "clockSource must not be null"); this.snapshotIntervalNanos = snapshotIntervalNanos; - tickUpdaterThread = new TickUpdaterThread(); + tickUpdaterThread = new TickUpdaterThread(snapshotIntervalNanos, clockSource, this::setSnapshotTickNanos); tickUpdaterThread.start(); } @@ -64,6 +61,10 @@ public long getTickNanos(boolean requestSnapshot) { return snapshotTickNanos; } + private void setSnapshotTickNanos(long snapshotTickNanos) { + this.snapshotTickNanos = snapshotTickNanos; + } + /** * A thread that updates snapshotTickNanos value periodically with a configured interval. * The thread is started when the DefaultMonotonicSnapshotClock is created and runs until the close method is @@ -73,24 +74,26 @@ public long getTickNanos(boolean requestSnapshot) { * stability of the read value since a single thread is scheduled on a single CPU. If the thread is migrated * to another CPU, the clock source value might leap backward or forward, but logic in this class will handle it. */ - private class TickUpdaterThread extends Thread { + private static class TickUpdaterThread extends Thread { private final Object tickUpdateDelayMonitor = new Object(); private final Object tickUpdatedMonitor = new Object(); - private final long maxDelta; - private long referenceClockSourceValue; - private long baseSnapshotTickNanos; - private long previousSnapshotTickNanos; + private final MonotonicLeapDetectingTickUpdater tickUpdater; private volatile boolean running; private boolean tickUpdateDelayMonitorNotified; private long requestCount; + private final long sleepMillis; + private final int sleepNanos; - TickUpdaterThread() { + TickUpdaterThread(long snapshotIntervalNanos, LongSupplier clockSource, LongConsumer setSnapshotTickNanos) { super(DefaultMonotonicSnapshotClock.class.getSimpleName() + "-update-loop"); // set as daemon thread so that it doesn't prevent the JVM from exiting setDaemon(true); // set the highest priority setPriority(MAX_PRIORITY); - this.maxDelta = 2 * snapshotIntervalNanos; + this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); + this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); + tickUpdater = new MonotonicLeapDetectingTickUpdater(clockSource, setSnapshotTickNanos, + 2 * snapshotIntervalNanos); } @Override @@ -100,7 +103,9 @@ public void run() { long updatedForRequestCount = -1; while (!isInterrupted()) { try { - boolean snapshotRequested = false; + // track if the thread has waited for the whole duration of the snapshot interval + // before updating the tick value + boolean waitedSnapshotInterval = false; // sleep for the configured interval on a monitor that can be notified to stop the sleep // and update the tick value immediately. This is used in requestUpdate method. synchronized (tickUpdateDelayMonitor) { @@ -109,11 +114,12 @@ public void run() { if (requestCount == updatedForRequestCount) { // if no request has been made, sleep for the configured interval tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); - snapshotRequested = tickUpdateDelayMonitorNotified; + waitedSnapshotInterval = !tickUpdateDelayMonitorNotified; } updatedForRequestCount = requestCount; } - updateSnapshotTickNanos(snapshotRequested); + // update the tick value using the tick updater which will tolerate leaps backward or forward + tickUpdater.update(waitedSnapshotInterval); notifyAllTickUpdated(); } catch (InterruptedException e) { interrupt(); @@ -131,44 +137,6 @@ public void run() { } } - private void updateSnapshotTickNanos(boolean snapshotRequested) { - long clockValue = clockSource.getAsLong(); - - // Initialization - if (referenceClockSourceValue == 0) { - referenceClockSourceValue = clockValue; - baseSnapshotTickNanos = clockValue; - snapshotTickNanos = clockValue; - previousSnapshotTickNanos = clockValue; - return; - } - - // calculate the duration since the reference clock source value - // so that the snapshot value is always increasing and tolerates it when the clock source is not strictly - // monotonic across all CPUs and leaps backward or forward - long durationSinceReference = clockValue - referenceClockSourceValue; - long newSnapshotTickNanos = baseSnapshotTickNanos + durationSinceReference; - - // reset the reference clock source value if the clock source value leaps backward or forward - if (newSnapshotTickNanos < previousSnapshotTickNanos - maxDelta - || newSnapshotTickNanos > previousSnapshotTickNanos + maxDelta) { - referenceClockSourceValue = clockValue; - baseSnapshotTickNanos = previousSnapshotTickNanos; - if (!snapshotRequested) { - // if the snapshot value is not requested, increment by the snapshot interval - baseSnapshotTickNanos += snapshotIntervalNanos; - } - newSnapshotTickNanos = baseSnapshotTickNanos; - } - - // update snapshotTickNanos value if the new value is greater than the previous value - if (newSnapshotTickNanos > previousSnapshotTickNanos) { - snapshotTickNanos = newSnapshotTickNanos; - // store into a field so that we don't need to do a volatile read to find out the previous value - previousSnapshotTickNanos = newSnapshotTickNanos; - } - } - private void notifyAllTickUpdated() { synchronized (tickUpdatedMonitor) { // notify all threads that are waiting for the tick value to be updated @@ -178,8 +146,8 @@ private void notifyAllTickUpdated() { public void requestUpdate() { if (!running) { - // thread has stopped running, fallback to update the value directly without any optimizations - snapshotTickNanos = clockSource.getAsLong(); + // thread has stopped running, fallback to update the value directly without optimizations + tickUpdater.update(false); return; } synchronized (tickUpdatedMonitor) { @@ -212,6 +180,73 @@ public synchronized void start() { } } + /** + * Handles updating the tick value in a monotonic way so that the value is always increasing, + * regardless of leaps backward and forward in the clock source value. + */ + private static class MonotonicLeapDetectingTickUpdater { + private final LongSupplier clockSource; + private final long snapshotInternalNanos; + private final long maxDeltaNanosForLeapDetection; + private final LongConsumer tickUpdatedCallback; + private long referenceClockSourceValue = Long.MIN_VALUE; + private long baseSnapshotTickNanos; + private long previousSnapshotTickNanos; + + MonotonicLeapDetectingTickUpdater(LongSupplier clockSource, LongConsumer tickUpdatedCallback, + long snapshotInternalNanos) { + this.clockSource = clockSource; + this.snapshotInternalNanos = snapshotInternalNanos; + this.maxDeltaNanosForLeapDetection = 2 * snapshotInternalNanos; + this.tickUpdatedCallback = tickUpdatedCallback; + } + + /** + * Updates the snapshot tick value. The tickUpdatedCallback is called if the value has changed. + * The value is updated in a monotonic way so that the value is always increasing, regardless of leaps backward + * and forward in the clock source value. + * Leap detection is done by comparing the new value with the previous value and the maximum delta value. + * + * @param waitedSnapshotInterval if true, the method has waited for the snapshot interval since the previous + * call. + */ + public synchronized void update(boolean waitedSnapshotInterval) { + long clockValue = clockSource.getAsLong(); + + // Initialization + if (referenceClockSourceValue == Long.MIN_VALUE) { + referenceClockSourceValue = clockValue; + baseSnapshotTickNanos = clockValue; + previousSnapshotTickNanos = clockValue; + tickUpdatedCallback.accept(clockValue); + return; + } + + // calculate the duration since the reference clock source value + // so that the snapshot value is always increasing and tolerates it when the clock source is not strictly + // monotonic across all CPUs and leaps backward or forward + long durationSinceReference = clockValue - referenceClockSourceValue; + long newSnapshotTickNanos = baseSnapshotTickNanos + durationSinceReference; + + // reset the reference clock source value if the clock source value leaps backward or forward + // more than the maximum delta value + if (newSnapshotTickNanos < previousSnapshotTickNanos - maxDeltaNanosForLeapDetection + || newSnapshotTickNanos > previousSnapshotTickNanos + maxDeltaNanosForLeapDetection) { + referenceClockSourceValue = clockValue; + long incrementWhenLeapDetected = waitedSnapshotInterval ? snapshotInternalNanos : 0; + baseSnapshotTickNanos = previousSnapshotTickNanos + incrementWhenLeapDetected; + newSnapshotTickNanos = baseSnapshotTickNanos; + } + + // update snapshotTickNanos value if the new value is greater than the previous value + if (newSnapshotTickNanos > previousSnapshotTickNanos) { + // store the previous value + previousSnapshotTickNanos = newSnapshotTickNanos; + tickUpdatedCallback.accept(newSnapshotTickNanos); + } + } + } + @Override public void close() { tickUpdaterThread.interrupt(); From d6d60f79e4671644b3379a0a4c2022e4340212f8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 12:26:15 +0200 Subject: [PATCH 10/63] Remove invalid test --- .../broker/qos/AsyncTokenBucketTest.java | 38 ------------------- 1 file changed, 38 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index d12f21c718cc5..89934803cf1ed 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -21,10 +21,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; -import java.util.Random; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import org.assertj.core.data.Offset; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -199,40 +197,4 @@ void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates // there should be 90 tokens available assertThat(asyncTokenBucket.tokens(true)).isEqualTo(90); } - - @Test - void shouldTolerateInstableClockSourceWhenUpdatingTokens() { - AtomicLong offset = new AtomicLong(0); - long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(100); - DefaultMonotonicSnapshotClock monotonicSnapshotClock = - new DefaultMonotonicSnapshotClock(resolutionNanos, - () -> offset.get() + manualClockSource.get()); - long initialTokens = 500L; - asyncTokenBucket = - AsyncTokenBucket.builder().resolutionNanos(resolutionNanos) - .capacity(100000).rate(1000).initialTokens(initialTokens).clock(monotonicSnapshotClock).build(); - Random random = new Random(0); - int randomOffsetCount = 0; - for (int i = 0; i < 100000; i++) { - // increment the clock by 1ms, since rate is 1000 tokens/s, this should make 1 token available - incrementMillis(1); - if (i % 39 == 0) { - // randomly offset the clock source - // update the tokens consistently before and after offsetting the clock source - asyncTokenBucket.tokens(true); - offset.set((random.nextBoolean() ? -1L : 1L) * random.nextLong(0L, 100L) * resolutionNanos); - asyncTokenBucket.tokens(true); - randomOffsetCount++; - } - // consume 1 token - asyncTokenBucket.consumeTokens(1); - } - assertThat(asyncTokenBucket.tokens(true)) - // since the rate is 1/ms and the test increments the clock by 1ms and consumes 1 token in each - // iteration, the tokens should be greater than or equal to the initial tokens - .isGreaterThanOrEqualTo(initialTokens) - // tolerate difference in added tokens since when clock leaps forward or backwards, the clock - // is assumed to have moved forward by the resolutionNanos - .isCloseTo(initialTokens, Offset.offset(3L * randomOffsetCount)); - } } \ No newline at end of file From 0886054950ee03c1177aab258306f14a97941de9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 14:07:14 +0200 Subject: [PATCH 11/63] Revert "Remove invalid test" This reverts commit d6d60f79e4671644b3379a0a4c2022e4340212f8. --- .../broker/qos/AsyncTokenBucketTest.java | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index 89934803cf1ed..d12f21c718cc5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -21,8 +21,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; +import java.util.Random; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import org.assertj.core.data.Offset; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -197,4 +199,40 @@ void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates // there should be 90 tokens available assertThat(asyncTokenBucket.tokens(true)).isEqualTo(90); } + + @Test + void shouldTolerateInstableClockSourceWhenUpdatingTokens() { + AtomicLong offset = new AtomicLong(0); + long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(100); + DefaultMonotonicSnapshotClock monotonicSnapshotClock = + new DefaultMonotonicSnapshotClock(resolutionNanos, + () -> offset.get() + manualClockSource.get()); + long initialTokens = 500L; + asyncTokenBucket = + AsyncTokenBucket.builder().resolutionNanos(resolutionNanos) + .capacity(100000).rate(1000).initialTokens(initialTokens).clock(monotonicSnapshotClock).build(); + Random random = new Random(0); + int randomOffsetCount = 0; + for (int i = 0; i < 100000; i++) { + // increment the clock by 1ms, since rate is 1000 tokens/s, this should make 1 token available + incrementMillis(1); + if (i % 39 == 0) { + // randomly offset the clock source + // update the tokens consistently before and after offsetting the clock source + asyncTokenBucket.tokens(true); + offset.set((random.nextBoolean() ? -1L : 1L) * random.nextLong(0L, 100L) * resolutionNanos); + asyncTokenBucket.tokens(true); + randomOffsetCount++; + } + // consume 1 token + asyncTokenBucket.consumeTokens(1); + } + assertThat(asyncTokenBucket.tokens(true)) + // since the rate is 1/ms and the test increments the clock by 1ms and consumes 1 token in each + // iteration, the tokens should be greater than or equal to the initial tokens + .isGreaterThanOrEqualTo(initialTokens) + // tolerate difference in added tokens since when clock leaps forward or backwards, the clock + // is assumed to have moved forward by the resolutionNanos + .isCloseTo(initialTokens, Offset.offset(3L * randomOffsetCount)); + } } \ No newline at end of file From f4feda1df2a37b94e253bfcfd5d531be32f8f0ca Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 14:59:42 +0200 Subject: [PATCH 12/63] Add test mode to DefaultMonotonicSnapshotClock so that thread updates don't cause problems in certain tests --- .../qos/DefaultMonotonicSnapshotClock.java | 37 +++++++++++++------ 1 file changed, 26 insertions(+), 11 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index b6f5de97f522d..ac825be19bea0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -37,18 +37,21 @@ */ public class DefaultMonotonicSnapshotClock implements MonotonicSnapshotClock, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(DefaultMonotonicSnapshotClock.class); - private final LongSupplier clockSource; private final TickUpdaterThread tickUpdaterThread; - private final long snapshotIntervalNanos; private volatile long snapshotTickNanos; public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier clockSource) { + this(snapshotIntervalNanos, clockSource, false); + } + + public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier clockSource, + boolean updateOnlyWhenRequested) { if (snapshotIntervalNanos < TimeUnit.MILLISECONDS.toNanos(1)) { throw new IllegalArgumentException("snapshotIntervalNanos must be at least 1 millisecond"); } - this.clockSource = Objects.requireNonNull(clockSource, "clockSource must not be null"); - this.snapshotIntervalNanos = snapshotIntervalNanos; - tickUpdaterThread = new TickUpdaterThread(snapshotIntervalNanos, clockSource, this::setSnapshotTickNanos); + tickUpdaterThread = new TickUpdaterThread(snapshotIntervalNanos, + Objects.requireNonNull(clockSource, "clockSource must not be null"), this::setSnapshotTickNanos, + updateOnlyWhenRequested); tickUpdaterThread.start(); } @@ -84,14 +87,20 @@ private static class TickUpdaterThread extends Thread { private final long sleepMillis; private final int sleepNanos; - TickUpdaterThread(long snapshotIntervalNanos, LongSupplier clockSource, LongConsumer setSnapshotTickNanos) { + TickUpdaterThread(long snapshotIntervalNanos, LongSupplier clockSource, LongConsumer setSnapshotTickNanos, + boolean updateOnlyWhenRequested) { super(DefaultMonotonicSnapshotClock.class.getSimpleName() + "-update-loop"); // set as daemon thread so that it doesn't prevent the JVM from exiting setDaemon(true); // set the highest priority setPriority(MAX_PRIORITY); - this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); - this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); + if (updateOnlyWhenRequested) { + this.sleepMillis = -1; + this.sleepNanos = -1; + } else { + this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); + this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); + } tickUpdater = new MonotonicLeapDetectingTickUpdater(clockSource, setSnapshotTickNanos, 2 * snapshotIntervalNanos); } @@ -112,8 +121,14 @@ public void run() { tickUpdateDelayMonitorNotified = false; // only wait if no explicit request has been made since the last update if (requestCount == updatedForRequestCount) { - // if no request has been made, sleep for the configured interval - tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); + if (sleepMillis > 0 || sleepNanos > 0) { + // if no request has been made, sleep for the configured interval + tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); + } else { + // when the sleepMillis is -1, the thread will wait indefinitely until notified. + // this is used only in testing with a test clock source that is manually updated. + tickUpdateDelayMonitor.wait(); + } waitedSnapshotInterval = !tickUpdateDelayMonitorNotified; } updatedForRequestCount = requestCount; @@ -168,9 +183,9 @@ public void requestUpdate() { @Override public synchronized void start() { - super.start(); // wait until the thread is started and the tick value has been updated synchronized (tickUpdatedMonitor) { + super.start(); try { tickUpdatedMonitor.wait(); } catch (InterruptedException e) { From 13aa2f7ebdf90180c8a016ac2131e81d43e9e5bc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 15:00:26 +0200 Subject: [PATCH 13/63] Use test mode to fix test --- .../pulsar/broker/qos/AsyncTokenBucketTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index d12f21c718cc5..37497c23b01d2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -203,25 +203,25 @@ void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates @Test void shouldTolerateInstableClockSourceWhenUpdatingTokens() { AtomicLong offset = new AtomicLong(0); - long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(100); + long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(1); DefaultMonotonicSnapshotClock monotonicSnapshotClock = new DefaultMonotonicSnapshotClock(resolutionNanos, - () -> offset.get() + manualClockSource.get()); + () -> offset.get() + manualClockSource.get(), true); long initialTokens = 500L; asyncTokenBucket = AsyncTokenBucket.builder().resolutionNanos(resolutionNanos) .capacity(100000).rate(1000).initialTokens(initialTokens).clock(monotonicSnapshotClock).build(); Random random = new Random(0); - int randomOffsetCount = 0; + long randomOffsetCount = 0; for (int i = 0; i < 100000; i++) { // increment the clock by 1ms, since rate is 1000 tokens/s, this should make 1 token available incrementMillis(1); + monotonicSnapshotClock.getTickNanos(true); if (i % 39 == 0) { // randomly offset the clock source // update the tokens consistently before and after offsetting the clock source - asyncTokenBucket.tokens(true); - offset.set((random.nextBoolean() ? -1L : 1L) * random.nextLong(0L, 100L) * resolutionNanos); - asyncTokenBucket.tokens(true); + offset.set((random.nextBoolean() ? -1L : 1L) * random.nextLong(4L, 100L) * resolutionNanos); + monotonicSnapshotClock.getTickNanos(true); randomOffsetCount++; } // consume 1 token @@ -233,6 +233,6 @@ void shouldTolerateInstableClockSourceWhenUpdatingTokens() { .isGreaterThanOrEqualTo(initialTokens) // tolerate difference in added tokens since when clock leaps forward or backwards, the clock // is assumed to have moved forward by the resolutionNanos - .isCloseTo(initialTokens, Offset.offset(3L * randomOffsetCount)); + .isCloseTo(initialTokens, Offset.offset(randomOffsetCount)); } } \ No newline at end of file From 8b36d716cc476005e88d5cdb9ce43e611c451d03 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 19:44:48 +0200 Subject: [PATCH 14/63] Improve testability --- .../qos/DefaultMonotonicSnapshotClock.java | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index ac825be19bea0..1c6b2492e3af4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.qos; +import com.google.common.annotations.VisibleForTesting; import java.util.Objects; import java.util.concurrent.TimeUnit; import java.util.function.LongConsumer; @@ -59,7 +60,7 @@ public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier cl @Override public long getTickNanos(boolean requestSnapshot) { if (requestSnapshot) { - tickUpdaterThread.requestUpdate(); + tickUpdaterThread.requestUpdateAndWait(); } return snapshotTickNanos; } @@ -68,6 +69,11 @@ private void setSnapshotTickNanos(long snapshotTickNanos) { this.snapshotTickNanos = snapshotTickNanos; } + @VisibleForTesting + public void requestUpdate() { + tickUpdaterThread.requestUpdate(); + } + /** * A thread that updates snapshotTickNanos value periodically with a configured interval. * The thread is started when the DefaultMonotonicSnapshotClock is created and runs until the close method is @@ -159,19 +165,14 @@ private void notifyAllTickUpdated() { } } - public void requestUpdate() { + public void requestUpdateAndWait() { if (!running) { // thread has stopped running, fallback to update the value directly without optimizations tickUpdater.update(false); return; } synchronized (tickUpdatedMonitor) { - // notify the thread to stop waiting and update the tick value - synchronized (tickUpdateDelayMonitor) { - tickUpdateDelayMonitorNotified = true; - requestCount++; - tickUpdateDelayMonitor.notify(); - } + requestUpdate(); // wait until the tick value has been updated try { tickUpdatedMonitor.wait(); @@ -181,6 +182,15 @@ public void requestUpdate() { } } + public void requestUpdate() { + // notify the thread to stop waiting and update the tick value + synchronized (tickUpdateDelayMonitor) { + tickUpdateDelayMonitorNotified = true; + requestCount++; + tickUpdateDelayMonitor.notify(); + } + } + @Override public synchronized void start() { // wait until the thread is started and the tick value has been updated From d9fb30fdaeff0e7dad8fa551d3c39e37f9924c09 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 19:45:27 +0200 Subject: [PATCH 15/63] Add failing test case --- .../DefaultMonotonicSnapshotClockTest.java | 41 ++++++++----------- 1 file changed, 16 insertions(+), 25 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java index 121522876ba65..54e92bf8c9d44 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java @@ -40,43 +40,34 @@ private static Object[] booleanValues() { @Test(dataProvider = "booleanValues") void testClockHandlesTimeLeapsBackwardsOrForward(boolean requestSnapshot) throws InterruptedException { long snapshotIntervalMillis = 5; - AtomicLong offsetValue = new AtomicLong(0); + AtomicLong clockValue = new AtomicLong(1); @Cleanup DefaultMonotonicSnapshotClock clock = new DefaultMonotonicSnapshotClock(Duration.ofMillis(snapshotIntervalMillis).toNanos(), - () -> System.nanoTime() + offsetValue.get()); + clockValue::get); + long previousTick = -1; boolean leapDirection = true; - for (int i = 0; i < 100; i++) { + for (int i = 0; i < 10000; i++) { + clockValue.addAndGet(TimeUnit.MILLISECONDS.toNanos(1)); + if (i % 5 == 0) { + clock.requestUpdate(); + } long tick = clock.getTickNanos(requestSnapshot); log.info("i = {}, tick = {}", i, tick); - if ((i + 1) % 3 == 0) { + if ((i + 1) % 5 == 0) { leapDirection = !leapDirection; log.info("Time leap 5 minutes {}", leapDirection ? "forward" : "backwards"); // make the clock leap 5 minute forward or backwards - offsetValue.set((leapDirection ? 1L : -1L) * Duration.ofMinutes(5).toNanos()); - Thread.sleep(2 * snapshotIntervalMillis); - } else { - Thread.sleep(snapshotIntervalMillis); + clockValue.addAndGet((leapDirection ? 1L : -1L) * Duration.ofMinutes(5).toNanos()); } - try { - var assertion = assertThat(tick) - .describedAs("i = %d, tick = %d, previousTick = %d", i, tick, previousTick); - if (requestSnapshot) { - assertion = assertion.isGreaterThan(previousTick); - } else { - assertion = assertion.isGreaterThanOrEqualTo(previousTick); - } - assertion.isCloseTo(previousTick, - Offset.offset(5 * TimeUnit.MILLISECONDS.toNanos(snapshotIntervalMillis))); - } catch (AssertionError e) { - if (i < 3) { - // ignore the assertion errors in first 3 rounds since classloading of AssertJ makes the timings - // flaky - } else { - throw e; - } + if (previousTick != -1) { + assertThat(tick) + .describedAs("i = %d, tick = %d, previousTick = %d", i, tick, previousTick) + .isGreaterThanOrEqualTo(previousTick) + .isCloseTo(previousTick, + Offset.offset(TimeUnit.MILLISECONDS.toNanos(1))); } previousTick = tick; } From 8c851e283e2da0d81628242ab956ae73203620d2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 20:24:44 +0200 Subject: [PATCH 16/63] Fix test --- .../broker/qos/DefaultMonotonicSnapshotClockTest.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java index 54e92bf8c9d44..cf7f889e5ba65 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java @@ -44,14 +44,15 @@ void testClockHandlesTimeLeapsBackwardsOrForward(boolean requestSnapshot) throws @Cleanup DefaultMonotonicSnapshotClock clock = new DefaultMonotonicSnapshotClock(Duration.ofMillis(snapshotIntervalMillis).toNanos(), - clockValue::get); + clockValue::get, true); long previousTick = -1; boolean leapDirection = true; for (int i = 0; i < 10000; i++) { clockValue.addAndGet(TimeUnit.MILLISECONDS.toNanos(1)); - if (i % 5 == 0) { + if (i % snapshotIntervalMillis == 0) { + // let the clock update by a background thread clock.requestUpdate(); } long tick = clock.getTickNanos(requestSnapshot); @@ -67,7 +68,11 @@ void testClockHandlesTimeLeapsBackwardsOrForward(boolean requestSnapshot) throws .describedAs("i = %d, tick = %d, previousTick = %d", i, tick, previousTick) .isGreaterThanOrEqualTo(previousTick) .isCloseTo(previousTick, - Offset.offset(TimeUnit.MILLISECONDS.toNanos(1))); + // then snapshot is requested, the time difference between the two ticks is accurate + // otherwise allow time difference at most 4 times the snapshot interval since the + // clock is updated periodically by a background thread + Offset.offset(TimeUnit.MILLISECONDS.toNanos( + requestSnapshot ? 1 : 4 * snapshotIntervalMillis))); } previousTick = tick; } From 846cc89adbbcc16fda287f94e8c07c9a9d6113f9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 20:24:54 +0200 Subject: [PATCH 17/63] Add failing test case for the negative tokens case --- .../broker/qos/AsyncTokenBucketTest.java | 21 +++++-------------- 1 file changed, 5 insertions(+), 16 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index 37497c23b01d2..f2bb4ebec6310 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -21,7 +21,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; -import java.util.Random; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import org.assertj.core.data.Offset; @@ -201,9 +200,9 @@ void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates } @Test - void shouldTolerateInstableClockSourceWhenUpdatingTokens() { + void shouldHandleEventualConsistency() { AtomicLong offset = new AtomicLong(0); - long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(1); + long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(100); DefaultMonotonicSnapshotClock monotonicSnapshotClock = new DefaultMonotonicSnapshotClock(resolutionNanos, () -> offset.get() + manualClockSource.get(), true); @@ -211,19 +210,11 @@ void shouldTolerateInstableClockSourceWhenUpdatingTokens() { asyncTokenBucket = AsyncTokenBucket.builder().resolutionNanos(resolutionNanos) .capacity(100000).rate(1000).initialTokens(initialTokens).clock(monotonicSnapshotClock).build(); - Random random = new Random(0); - long randomOffsetCount = 0; for (int i = 0; i < 100000; i++) { // increment the clock by 1ms, since rate is 1000 tokens/s, this should make 1 token available incrementMillis(1); - monotonicSnapshotClock.getTickNanos(true); - if (i % 39 == 0) { - // randomly offset the clock source - // update the tokens consistently before and after offsetting the clock source - offset.set((random.nextBoolean() ? -1L : 1L) * random.nextLong(4L, 100L) * resolutionNanos); - monotonicSnapshotClock.getTickNanos(true); - randomOffsetCount++; - } + // request the clock to be updated + monotonicSnapshotClock.requestUpdate(); // consume 1 token asyncTokenBucket.consumeTokens(1); } @@ -231,8 +222,6 @@ void shouldTolerateInstableClockSourceWhenUpdatingTokens() { // since the rate is 1/ms and the test increments the clock by 1ms and consumes 1 token in each // iteration, the tokens should be greater than or equal to the initial tokens .isGreaterThanOrEqualTo(initialTokens) - // tolerate difference in added tokens since when clock leaps forward or backwards, the clock - // is assumed to have moved forward by the resolutionNanos - .isCloseTo(initialTokens, Offset.offset(randomOffsetCount)); + .isCloseTo(initialTokens, Offset.offset(1000L)); } } \ No newline at end of file From 58f8b3509727dab6acb1fcdb8ef1ac04d67ba677 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Feb 2025 22:18:09 +0200 Subject: [PATCH 18/63] Don't handle leaps forward since those cannot be detected properly --- .../broker/qos/DefaultMonotonicSnapshotClock.java | 13 ++++++------- .../pulsar/broker/qos/AsyncTokenBucketTest.java | 4 +--- .../qos/DefaultMonotonicSnapshotClockTest.java | 13 ++++--------- 3 files changed, 11 insertions(+), 19 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index 1c6b2492e3af4..0eadf4e25df83 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -139,7 +139,7 @@ public void run() { } updatedForRequestCount = requestCount; } - // update the tick value using the tick updater which will tolerate leaps backward or forward + // update the tick value using the tick updater which will tolerate leaps backward tickUpdater.update(waitedSnapshotInterval); notifyAllTickUpdated(); } catch (InterruptedException e) { @@ -207,7 +207,7 @@ public synchronized void start() { /** * Handles updating the tick value in a monotonic way so that the value is always increasing, - * regardless of leaps backward and forward in the clock source value. + * regardless of leaps backward in the clock source value. */ private static class MonotonicLeapDetectingTickUpdater { private final LongSupplier clockSource; @@ -229,7 +229,7 @@ private static class MonotonicLeapDetectingTickUpdater { /** * Updates the snapshot tick value. The tickUpdatedCallback is called if the value has changed. * The value is updated in a monotonic way so that the value is always increasing, regardless of leaps backward - * and forward in the clock source value. + * in the clock source value. * Leap detection is done by comparing the new value with the previous value and the maximum delta value. * * @param waitedSnapshotInterval if true, the method has waited for the snapshot interval since the previous @@ -249,14 +249,13 @@ public synchronized void update(boolean waitedSnapshotInterval) { // calculate the duration since the reference clock source value // so that the snapshot value is always increasing and tolerates it when the clock source is not strictly - // monotonic across all CPUs and leaps backward or forward + // monotonic across all CPUs and leaps backward long durationSinceReference = clockValue - referenceClockSourceValue; long newSnapshotTickNanos = baseSnapshotTickNanos + durationSinceReference; - // reset the reference clock source value if the clock source value leaps backward or forward + // reset the reference clock source value if the clock source value leaps backward // more than the maximum delta value - if (newSnapshotTickNanos < previousSnapshotTickNanos - maxDeltaNanosForLeapDetection - || newSnapshotTickNanos > previousSnapshotTickNanos + maxDeltaNanosForLeapDetection) { + if (newSnapshotTickNanos < previousSnapshotTickNanos - maxDeltaNanosForLeapDetection) { referenceClockSourceValue = clockValue; long incrementWhenLeapDetected = waitedSnapshotInterval ? snapshotInternalNanos : 0; baseSnapshotTickNanos = previousSnapshotTickNanos + incrementWhenLeapDetected; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index f2bb4ebec6310..3bfa8238d86a3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -205,7 +205,7 @@ void shouldHandleEventualConsistency() { long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(100); DefaultMonotonicSnapshotClock monotonicSnapshotClock = new DefaultMonotonicSnapshotClock(resolutionNanos, - () -> offset.get() + manualClockSource.get(), true); + () -> offset.get() + manualClockSource.get()); long initialTokens = 500L; asyncTokenBucket = AsyncTokenBucket.builder().resolutionNanos(resolutionNanos) @@ -213,8 +213,6 @@ void shouldHandleEventualConsistency() { for (int i = 0; i < 100000; i++) { // increment the clock by 1ms, since rate is 1000 tokens/s, this should make 1 token available incrementMillis(1); - // request the clock to be updated - monotonicSnapshotClock.requestUpdate(); // consume 1 token asyncTokenBucket.consumeTokens(1); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java index cf7f889e5ba65..7d99da5f67bb4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java @@ -38,30 +38,25 @@ private static Object[] booleanValues() { } @Test(dataProvider = "booleanValues") - void testClockHandlesTimeLeapsBackwardsOrForward(boolean requestSnapshot) throws InterruptedException { + void testClockHandlesTimeLeapsBackwards(boolean requestSnapshot) throws InterruptedException { long snapshotIntervalMillis = 5; AtomicLong clockValue = new AtomicLong(1); @Cleanup DefaultMonotonicSnapshotClock clock = new DefaultMonotonicSnapshotClock(Duration.ofMillis(snapshotIntervalMillis).toNanos(), - clockValue::get, true); + clockValue::get); long previousTick = -1; boolean leapDirection = true; for (int i = 0; i < 10000; i++) { clockValue.addAndGet(TimeUnit.MILLISECONDS.toNanos(1)); - if (i % snapshotIntervalMillis == 0) { - // let the clock update by a background thread - clock.requestUpdate(); - } long tick = clock.getTickNanos(requestSnapshot); log.info("i = {}, tick = {}", i, tick); if ((i + 1) % 5 == 0) { leapDirection = !leapDirection; - log.info("Time leap 5 minutes {}", leapDirection ? "forward" : "backwards"); - // make the clock leap 5 minute forward or backwards - clockValue.addAndGet((leapDirection ? 1L : -1L) * Duration.ofMinutes(5).toNanos()); + log.info("Time leap 5 minutes backwards"); + clockValue.addAndGet(-Duration.ofMinutes(5).toNanos()); } if (previousTick != -1) { assertThat(tick) From 91ae4c6aaec1d2a604a342a0b5ed743806aea467 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 09:55:17 +0200 Subject: [PATCH 19/63] Remove separate test mode since it's not needed --- .../qos/DefaultMonotonicSnapshotClock.java | 30 ++++--------------- 1 file changed, 6 insertions(+), 24 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index 0eadf4e25df83..339621292c4d8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -42,17 +42,11 @@ public class DefaultMonotonicSnapshotClock implements MonotonicSnapshotClock, Au private volatile long snapshotTickNanos; public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier clockSource) { - this(snapshotIntervalNanos, clockSource, false); - } - - public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier clockSource, - boolean updateOnlyWhenRequested) { if (snapshotIntervalNanos < TimeUnit.MILLISECONDS.toNanos(1)) { throw new IllegalArgumentException("snapshotIntervalNanos must be at least 1 millisecond"); } tickUpdaterThread = new TickUpdaterThread(snapshotIntervalNanos, - Objects.requireNonNull(clockSource, "clockSource must not be null"), this::setSnapshotTickNanos, - updateOnlyWhenRequested); + Objects.requireNonNull(clockSource, "clockSource must not be null"), this::setSnapshotTickNanos); tickUpdaterThread.start(); } @@ -93,20 +87,14 @@ private static class TickUpdaterThread extends Thread { private final long sleepMillis; private final int sleepNanos; - TickUpdaterThread(long snapshotIntervalNanos, LongSupplier clockSource, LongConsumer setSnapshotTickNanos, - boolean updateOnlyWhenRequested) { + TickUpdaterThread(long snapshotIntervalNanos, LongSupplier clockSource, LongConsumer setSnapshotTickNanos) { super(DefaultMonotonicSnapshotClock.class.getSimpleName() + "-update-loop"); // set as daemon thread so that it doesn't prevent the JVM from exiting setDaemon(true); // set the highest priority setPriority(MAX_PRIORITY); - if (updateOnlyWhenRequested) { - this.sleepMillis = -1; - this.sleepNanos = -1; - } else { - this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); - this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); - } + this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); + this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); tickUpdater = new MonotonicLeapDetectingTickUpdater(clockSource, setSnapshotTickNanos, 2 * snapshotIntervalNanos); } @@ -127,14 +115,8 @@ public void run() { tickUpdateDelayMonitorNotified = false; // only wait if no explicit request has been made since the last update if (requestCount == updatedForRequestCount) { - if (sleepMillis > 0 || sleepNanos > 0) { - // if no request has been made, sleep for the configured interval - tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); - } else { - // when the sleepMillis is -1, the thread will wait indefinitely until notified. - // this is used only in testing with a test clock source that is manually updated. - tickUpdateDelayMonitor.wait(); - } + // if no request has been made, sleep for the configured interval + tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); waitedSnapshotInterval = !tickUpdateDelayMonitorNotified; } updatedForRequestCount = requestCount; From 487419e5f56de37cde16223feba3b9ea0d91b991 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 09:58:02 +0200 Subject: [PATCH 20/63] Fix parameter --- .../apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index 339621292c4d8..b17ce5f601614 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -96,7 +96,7 @@ private static class TickUpdaterThread extends Thread { this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); tickUpdater = new MonotonicLeapDetectingTickUpdater(clockSource, setSnapshotTickNanos, - 2 * snapshotIntervalNanos); + snapshotIntervalNanos); } @Override From 571b2c2f75718b38fbe84e962f2fa7da391ccdda Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 10:04:22 +0200 Subject: [PATCH 21/63] Enable batching of update requests by using a AtomicLong for the requestCount --- .../broker/qos/DefaultMonotonicSnapshotClock.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index b17ce5f601614..962ad0980fe86 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import java.util.Objects; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.LongConsumer; import java.util.function.LongSupplier; import org.slf4j.Logger; @@ -83,7 +84,7 @@ private static class TickUpdaterThread extends Thread { private final MonotonicLeapDetectingTickUpdater tickUpdater; private volatile boolean running; private boolean tickUpdateDelayMonitorNotified; - private long requestCount; + private AtomicLong requestCount = new AtomicLong(); private final long sleepMillis; private final int sleepNanos; @@ -114,13 +115,13 @@ public void run() { synchronized (tickUpdateDelayMonitor) { tickUpdateDelayMonitorNotified = false; // only wait if no explicit request has been made since the last update - if (requestCount == updatedForRequestCount) { + if (requestCount.get() == updatedForRequestCount) { // if no request has been made, sleep for the configured interval tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); waitedSnapshotInterval = !tickUpdateDelayMonitorNotified; } - updatedForRequestCount = requestCount; } + updatedForRequestCount = requestCount.get(); // update the tick value using the tick updater which will tolerate leaps backward tickUpdater.update(waitedSnapshotInterval); notifyAllTickUpdated(); @@ -165,10 +166,12 @@ public void requestUpdateAndWait() { } public void requestUpdate() { + // increment the request count that ensures that the thread will update the tick value after this request + // was made also when there's a race condition between the request and the update + requestCount.incrementAndGet(); // notify the thread to stop waiting and update the tick value synchronized (tickUpdateDelayMonitor) { tickUpdateDelayMonitorNotified = true; - requestCount++; tickUpdateDelayMonitor.notify(); } } From 532998d3df2f959ec291f6e17feb91be06ae7af8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 10:07:38 +0200 Subject: [PATCH 22/63] Removing remaining parts of the test mode --- .../pulsar/broker/qos/DefaultMonotonicSnapshotClock.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index 962ad0980fe86..d1df46b3cab2c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.qos; -import com.google.common.annotations.VisibleForTesting; import java.util.Objects; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -64,11 +63,6 @@ private void setSnapshotTickNanos(long snapshotTickNanos) { this.snapshotTickNanos = snapshotTickNanos; } - @VisibleForTesting - public void requestUpdate() { - tickUpdaterThread.requestUpdate(); - } - /** * A thread that updates snapshotTickNanos value periodically with a configured interval. * The thread is started when the DefaultMonotonicSnapshotClock is created and runs until the close method is @@ -165,7 +159,7 @@ public void requestUpdateAndWait() { } } - public void requestUpdate() { + private void requestUpdate() { // increment the request count that ensures that the thread will update the tick value after this request // was made also when there's a race condition between the request and the update requestCount.incrementAndGet(); From a21eee3a995fc0ca2e3690a1f3259828b006f549 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 10:11:23 +0200 Subject: [PATCH 23/63] Remove synchronization from tick updater since the usual path is single threaded --- .../pulsar/broker/qos/DefaultMonotonicSnapshotClock.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index d1df46b3cab2c..09566d11f2b64 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -144,8 +144,10 @@ private void notifyAllTickUpdated() { public void requestUpdateAndWait() { if (!running) { - // thread has stopped running, fallback to update the value directly without optimizations - tickUpdater.update(false); + synchronized (tickUpdater) { + // thread has stopped running, fallback to update the value directly without optimizations + tickUpdater.update(false); + } return; } synchronized (tickUpdatedMonitor) { @@ -214,7 +216,7 @@ private static class MonotonicLeapDetectingTickUpdater { * @param waitedSnapshotInterval if true, the method has waited for the snapshot interval since the previous * call. */ - public synchronized void update(boolean waitedSnapshotInterval) { + public void update(boolean waitedSnapshotInterval) { long clockValue = clockSource.getAsLong(); // Initialization From eb82ab98e0508f2c16f859bec347a9b0c15c2828 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 10:12:24 +0200 Subject: [PATCH 24/63] Improve comments --- .../pulsar/broker/qos/DefaultMonotonicSnapshotClock.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index 09566d11f2b64..50cc4fa5ca947 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -217,13 +217,15 @@ private static class MonotonicLeapDetectingTickUpdater { * call. */ public void update(boolean waitedSnapshotInterval) { + // get the current clock source value long clockValue = clockSource.getAsLong(); - // Initialization + // Initialization on first call if (referenceClockSourceValue == Long.MIN_VALUE) { referenceClockSourceValue = clockValue; baseSnapshotTickNanos = clockValue; previousSnapshotTickNanos = clockValue; + // update the tick value using the callback tickUpdatedCallback.accept(clockValue); return; } @@ -247,6 +249,7 @@ public void update(boolean waitedSnapshotInterval) { if (newSnapshotTickNanos > previousSnapshotTickNanos) { // store the previous value previousSnapshotTickNanos = newSnapshotTickNanos; + // update the tick value using the callback tickUpdatedCallback.accept(newSnapshotTickNanos); } } From d1dd4b745f38c6f9b7bd449f190c8bbf993c3559 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 10:18:15 +0200 Subject: [PATCH 25/63] Reorder methods --- .../qos/DefaultMonotonicSnapshotClock.java | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index 50cc4fa5ca947..d914078d9cc1c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -50,6 +50,10 @@ public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier cl tickUpdaterThread.start(); } + private void setSnapshotTickNanos(long snapshotTickNanos) { + this.snapshotTickNanos = snapshotTickNanos; + } + /** {@inheritDoc} */ @Override public long getTickNanos(boolean requestSnapshot) { @@ -59,8 +63,9 @@ public long getTickNanos(boolean requestSnapshot) { return snapshotTickNanos; } - private void setSnapshotTickNanos(long snapshotTickNanos) { - this.snapshotTickNanos = snapshotTickNanos; + @Override + public void close() { + tickUpdaterThread.interrupt(); } /** @@ -234,14 +239,22 @@ public void update(boolean waitedSnapshotInterval) { // so that the snapshot value is always increasing and tolerates it when the clock source is not strictly // monotonic across all CPUs and leaps backward long durationSinceReference = clockValue - referenceClockSourceValue; + // calculate the new snapshot tick value as a duration since the reference clock source value + // and add it to the base snapshot tick value long newSnapshotTickNanos = baseSnapshotTickNanos + durationSinceReference; // reset the reference clock source value if the clock source value leaps backward // more than the maximum delta value if (newSnapshotTickNanos < previousSnapshotTickNanos - maxDeltaNanosForLeapDetection) { + // when the clock source value leaps backward, reset the reference value to the new value + // for future duration calculations referenceClockSourceValue = clockValue; + // if the updater thread has waited for the snapshot interval since the previous call, + // increment the base snapshot tick value by the snapshot interval value long incrementWhenLeapDetected = waitedSnapshotInterval ? snapshotInternalNanos : 0; + // set the base snapshot tick value to the new value baseSnapshotTickNanos = previousSnapshotTickNanos + incrementWhenLeapDetected; + // set the new snapshot tick value to the base value newSnapshotTickNanos = baseSnapshotTickNanos; } @@ -254,9 +267,4 @@ public void update(boolean waitedSnapshotInterval) { } } } - - @Override - public void close() { - tickUpdaterThread.interrupt(); - } } From d5ea482ac62b3cb4521b7c319aed752f2fd9f155 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 11:25:40 +0200 Subject: [PATCH 26/63] Improve javadoc --- .../qos/DefaultMonotonicSnapshotClock.java | 30 +++++++++++++++---- 1 file changed, 24 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index d914078d9cc1c..3fcae31ca13b8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -28,13 +28,31 @@ import org.slf4j.LoggerFactory; /** - * Default implementation of {@link MonotonicSnapshotClock}. + * Default implementation of {@link MonotonicSnapshotClock} optimized for use with {@link AsyncTokenBucket}. * - * Starts a daemon thread that updates the snapshot value periodically with a configured interval. The close method - * should be called to stop the thread. - * A single thread is used to update the monotonic clock value so that the snapshot value is always increasing, - * even if the clock source is not strictly monotonic across all CPUs. This might be the case in some virtualized - * environments. + *

+ * This class provides a monotonic snapshot value that consistently increases, ensuring reliable behavior + * even in environments where the underlying clock source may not be strictly monotonic across all CPUs, + * such as certain virtualized platforms. + *

+ * + *

+ * Upon instantiation, a daemon thread is launched to periodically update the snapshot value at a configured + * interval. It is essential to invoke the {@link #close()} method to gracefully terminate this thread when it is + * no longer needed. + *

+ * + *

+ * The {@link AsyncTokenBucket} utilizes this clock to obtain tick values. It does not require a consistent value on + * every retrieval. However, when a consistent snapshot is necessary, the {@link #getTickNanos(boolean)} method + * is called with the {@code requestSnapshot} parameter set to {@code true}. + *

+ * + *

+ * By employing a single thread to update the monotonic clock value, this implementation ensures that the snapshot + * value remains strictly increasing. This approach mitigates potential inconsistencies that may arise from clock + * source discrepancies across different CPUs. + *

*/ public class DefaultMonotonicSnapshotClock implements MonotonicSnapshotClock, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(DefaultMonotonicSnapshotClock.class); From b19e0de62e163b3205da7aa24ff9694babebb4ae Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 11:25:46 +0200 Subject: [PATCH 27/63] Reduce excessive logging in test --- .../pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java index 7d99da5f67bb4..0e4b1b4a51446 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java @@ -52,10 +52,10 @@ void testClockHandlesTimeLeapsBackwards(boolean requestSnapshot) throws Interrup for (int i = 0; i < 10000; i++) { clockValue.addAndGet(TimeUnit.MILLISECONDS.toNanos(1)); long tick = clock.getTickNanos(requestSnapshot); - log.info("i = {}, tick = {}", i, tick); + //log.info("i = {}, tick = {}", i, tick); if ((i + 1) % 5 == 0) { leapDirection = !leapDirection; - log.info("Time leap 5 minutes backwards"); + //log.info("Time leap 5 minutes backwards"); clockValue.addAndGet(-Duration.ofMinutes(5).toNanos()); } if (previousTick != -1) { From b46fbab17d6fded5a26ab42b9aa93fb2457d92be Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 11:32:56 +0200 Subject: [PATCH 28/63] Reduce duplication in JMH test --- .../pulsar/broker/qos/AsyncTokenBucketBenchmark.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java b/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java index 23a9bfe8348bd..1b210258f13d2 100644 --- a/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java +++ b/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java @@ -61,8 +61,7 @@ public void teardown() { @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) public void consumeTokensBenchmark001Threads(Blackhole blackhole) { - asyncTokenBucket.consumeTokens(1); - blackhole.consume(asyncTokenBucket.getTokens()); + consumeTokenAndGetTokens(blackhole); } @Threads(10) @@ -70,8 +69,7 @@ public void consumeTokensBenchmark001Threads(Blackhole blackhole) { @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) public void consumeTokensBenchmark010Threads(Blackhole blackhole) { - asyncTokenBucket.consumeTokens(1); - blackhole.consume(asyncTokenBucket.getTokens()); + consumeTokenAndGetTokens(blackhole); } @Threads(100) @@ -79,7 +77,12 @@ public void consumeTokensBenchmark010Threads(Blackhole blackhole) { @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) public void consumeTokensBenchmark100Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole); + } + + private void consumeTokenAndGetTokens(Blackhole blackhole) { asyncTokenBucket.consumeTokens(1); + // blackhole is used to ensure that the compiler doesn't do dead code elimination blackhole.consume(asyncTokenBucket.getTokens()); } } From 73c2b8c25efaebc8d4b565f87a053ecaef7f9989 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 11:38:04 +0200 Subject: [PATCH 29/63] Add JMH benchmark for DefaultMonotonicSnapshotClock --- ...efaultMonotonicSnapshotClockBenchmark.java | 102 ++++++++++++++++++ 1 file changed, 102 insertions(+) create mode 100644 microbench/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockBenchmark.java diff --git a/microbench/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockBenchmark.java b/microbench/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockBenchmark.java new file mode 100644 index 0000000000000..d9054b8fe4be8 --- /dev/null +++ b/microbench/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockBenchmark.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +import java.util.concurrent.TimeUnit; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(3) +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.SECONDS) +@State(Scope.Thread) +public class DefaultMonotonicSnapshotClockBenchmark { + private DefaultMonotonicSnapshotClock monotonicSnapshotClock = + new DefaultMonotonicSnapshotClock(TimeUnit.MILLISECONDS.toNanos(1), System::nanoTime); + + @TearDown(Level.Iteration) + public void teardown() { + monotonicSnapshotClock.close(); + } + + @Threads(1) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanos001Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, false); + } + + @Threads(10) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanos010Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, false); + } + + @Threads(100) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanos100Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, false); + } + + @Threads(1) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanosRequestSnapshot001Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, true); + } + + @Threads(10) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanosRequestSnapshot010Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, true); + } + + @Threads(100) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanosRequestSnapshot100Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, true); + } + + private void consumeTokenAndGetTokens(Blackhole blackhole, boolean requestSnapshot) { + // blackhole is used to ensure that the compiler doesn't do dead code elimination + blackhole.consume(monotonicSnapshotClock.getTickNanos(requestSnapshot)); + } +} From a22da98f6f020a97e22fcdb1e6b74e3052f52713 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 11:50:15 +0200 Subject: [PATCH 30/63] Add more instructions for running JMH benchmarks --- microbench/README.md | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/microbench/README.md b/microbench/README.md index 780e3a5a1d3e8..f50c3036ff494 100644 --- a/microbench/README.md +++ b/microbench/README.md @@ -41,3 +41,29 @@ For fast recompiling of the benchmarks (without compiling Pulsar modules) and cr mvn -Pmicrobench -pl microbench clean package ``` +### Running specific benchmarks + +Display help: + +```shell +java -jar microbench/target/microbenchmarks.jar -h +``` + +Listing all benchmarks: + +```shell +java -jar microbench/target/microbenchmarks.jar -l +``` + +Running specific benchmarks: + +```shell +java -jar microbench/target/microbenchmarks.jar ".*BenchmarkName.*" +``` + +Checking what benchmarks match the pattern: + +```shell +java -jar microbench/target/microbenchmarks.jar ".*BenchmarkName.*" -lp +``` + From 428783882115f4d3d30353609b45e3d06017916e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 13:13:19 +0200 Subject: [PATCH 31/63] Add unit tests for leap detection --- .../qos/DefaultMonotonicSnapshotClock.java | 2 +- .../DefaultMonotonicSnapshotClockTest.java | 63 +++++++++++++++++++ 2 files changed, 64 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index 3fcae31ca13b8..0a8379ca80784 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -213,7 +213,7 @@ public synchronized void start() { * Handles updating the tick value in a monotonic way so that the value is always increasing, * regardless of leaps backward in the clock source value. */ - private static class MonotonicLeapDetectingTickUpdater { + static class MonotonicLeapDetectingTickUpdater { private final LongSupplier clockSource; private final long snapshotInternalNanos; private final long maxDeltaNanosForLeapDetection; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java index 0e4b1b4a51446..7e29b3e58f663 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java @@ -119,4 +119,67 @@ void testFailureHandlingInClockSource() { .isInstanceOf(RuntimeException.class) .hasMessage("Test clock failure"); } + + @Test + void testLeapDetectionIndepently() { + AtomicLong clockValue = new AtomicLong(0); + AtomicLong tickValue = new AtomicLong(0); + long expectedTickValue = 0; + long snapshotIntervalNanos = TimeUnit.MILLISECONDS.toNanos(1); + DefaultMonotonicSnapshotClock.MonotonicLeapDetectingTickUpdater updater = + new DefaultMonotonicSnapshotClock.MonotonicLeapDetectingTickUpdater(clockValue::get, tickValue::set, + snapshotIntervalNanos); + + updater.update(true); + + // advance the clock + clockValue.addAndGet(snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // simulate a leap backwards in time + clockValue.addAndGet(-10 * snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // advance the clock + clockValue.addAndGet(snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // simulate a leap backwards in time, without waiting a full snapshot interval + clockValue.addAndGet(-10 * snapshotIntervalNanos); + updater.update(false); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // advance the clock + clockValue.addAndGet(snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // simulate a small leap backwards in time which isn't detected, without waiting a full snapshot interval + clockValue.addAndGet(-1 * snapshotIntervalNanos); + updater.update(false); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + // clock doesn't advance for one snapshot interval + clockValue.addAndGet(snapshotIntervalNanos); + updater.update(false); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + // now the clock should advance again + clockValue.addAndGet(snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(false); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // simulate a leap forward + clockValue.addAndGet(10 * snapshotIntervalNanos); + // no special handling for leap forward + expectedTickValue += 10 * snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + } } \ No newline at end of file From 12cb40041611480caf88c285e608949afd15c077 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 13:26:42 +0200 Subject: [PATCH 32/63] Move updating of requestCount outside of the synchronization blocks to batch more requests together --- .../qos/DefaultMonotonicSnapshotClock.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index 0a8379ca80784..367074d39904c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -173,8 +173,15 @@ public void requestUpdateAndWait() { } return; } + // increment the request count that ensures that the thread will update the tick value after this request + // was made also when there's a race condition between the request and the update + requestCount.incrementAndGet(); synchronized (tickUpdatedMonitor) { - requestUpdate(); + // notify the thread to stop waiting and update the tick value + synchronized (tickUpdateDelayMonitor) { + tickUpdateDelayMonitorNotified = true; + tickUpdateDelayMonitor.notify(); + } // wait until the tick value has been updated try { tickUpdatedMonitor.wait(); @@ -184,17 +191,6 @@ public void requestUpdateAndWait() { } } - private void requestUpdate() { - // increment the request count that ensures that the thread will update the tick value after this request - // was made also when there's a race condition between the request and the update - requestCount.incrementAndGet(); - // notify the thread to stop waiting and update the tick value - synchronized (tickUpdateDelayMonitor) { - tickUpdateDelayMonitorNotified = true; - tickUpdateDelayMonitor.notify(); - } - } - @Override public synchronized void start() { // wait until the thread is started and the tick value has been updated From 31a1b30ed9b355f098eb24ab9d80089e61e0d37c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 14:01:54 +0200 Subject: [PATCH 33/63] Fix typo and improve comment --- .../pulsar/broker/qos/DefaultMonotonicSnapshotClock.java | 3 +++ .../pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index 367074d39904c..23b9359c8042d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -175,6 +175,9 @@ public void requestUpdateAndWait() { } // increment the request count that ensures that the thread will update the tick value after this request // was made also when there's a race condition between the request and the update + // this solution doesn't prevent all races, and it's not guaranteed that the tick value is always updated + // it will prevent the request having to wait for the delayed update cycle. This is sufficient for the + // use case. requestCount.incrementAndGet(); synchronized (tickUpdatedMonitor) { // notify the thread to stop waiting and update the tick value diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java index 7e29b3e58f663..0820b439915bb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java @@ -121,7 +121,7 @@ void testFailureHandlingInClockSource() { } @Test - void testLeapDetectionIndepently() { + void testLeapDetectionIndependently() { AtomicLong clockValue = new AtomicLong(0); AtomicLong tickValue = new AtomicLong(0); long expectedTickValue = 0; From 6e4a94fa728801941a7736557dec55ec8e591272 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 16:06:47 +0200 Subject: [PATCH 34/63] Optimize token calculation performance and correctness - since the AsyncTokenBucket it lockless, there could be a race condition where one thread adds tokens and another thread also adds tokens immediately after, and it also flushes the pending consumed tokens before the first thread executes that part. The potential problem is in the calculation where the new tokens get capped and the pending tokens get subtracted from the actual value. This could also lead to unnecessary negative tokens when the DefaultMonotonicSnapshotClock thread is starved and the tick update was delayed. This change also improves performance since there would be less contention in flushing the pending tokens and updating the token value. The changes limit the number of updates with the conditional logic in LAST_NANOS_UPDATER --- .../pulsar/broker/qos/AsyncTokenBucket.java | 34 ++++++++++++++----- .../broker/qos/AsyncTokenBucketTest.java | 22 +++++++++--- 2 files changed, 43 insertions(+), 13 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index 36edb1c5d8a9e..fb35e129a5e72 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -122,6 +122,7 @@ public static void resetToDefaultEventualConsistentTokensView() { protected AsyncTokenBucket(MonotonicSnapshotClock clockSource, long resolutionNanos) { this.clockSource = clockSource; this.resolutionNanos = resolutionNanos; + this.lastNanos = Long.MIN_VALUE; } public static FinalRateAsyncTokenBucketBuilder builder() { @@ -155,18 +156,22 @@ private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolea throw new IllegalArgumentException("consumeTokens must be >= 0"); } long currentNanos = clockSource.getTickNanos(forceUpdateTokens); + long newTokens = 0; // check if the tokens should be updated immediately if (shouldUpdateTokensImmediately(currentNanos, forceUpdateTokens)) { // calculate the number of new tokens since the last update - long newTokens = calculateNewTokensSinceLastUpdate(currentNanos); + newTokens = calculateNewTokensSinceLastUpdate(currentNanos); + } + if (newTokens > 0) { // flush the pendingConsumedTokens by calling "sumThenReset" long currentPendingConsumedTokens = pendingConsumedTokens.sumThenReset(); + // calculate the token delta by subtracting the consumed tokens from the new tokens + long tokenDelta = newTokens - currentPendingConsumedTokens; // update the tokens and return the current token value return TOKENS_UPDATER.updateAndGet(this, - // after adding new tokens subtract the pending consumed tokens and // limit the tokens to the capacity of the bucket - currentTokens -> Math.min(currentTokens + newTokens - currentPendingConsumedTokens, getCapacity()) - // subtract the consumed tokens + currentTokens -> Math.min(currentTokens + tokenDelta, getCapacity()) + // subtract the consumed tokens from the capped tokens - consumeTokens); } else { // eventual consistent fast path, tokens are not updated immediately @@ -176,8 +181,13 @@ private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolea pendingConsumedTokens.add(consumeTokens); } - // return Long.MIN_VALUE if the current value of tokens is unknown due to the eventual consistency - return Long.MIN_VALUE; + if (forceUpdateTokens) { + // return the current tokens balance without updating the tokens and resetting the pendingConsumedTokens + return tokens - pendingConsumedTokens.sum(); + } else { + // return Long.MIN_VALUE if the current value of tokens is unknown due to the eventual consistency + return Long.MIN_VALUE; + } } } @@ -210,10 +220,16 @@ private boolean shouldUpdateTokensImmediately(long currentNanos, boolean forceUp */ private long calculateNewTokensSinceLastUpdate(long currentNanos) { long newTokens; - // update lastNanos if currentNanos is greater than the current lastNanos long previousLastNanos = LAST_NANOS_UPDATER.getAndUpdate(this, - currentLastNanos -> Math.max(currentNanos, currentLastNanos)); - if (previousLastNanos == 0 || previousLastNanos >= currentNanos) { + currentLastNanos -> { + // update lastNanos only if at least resolutionNanos/2 nanoseconds has passed since the last update + if (currentNanos >= currentLastNanos + resolutionNanos / 2) { + return currentNanos; + } else { + return currentLastNanos; + } + }); + if (previousLastNanos == Long.MIN_VALUE || previousLastNanos >= currentNanos) { newTokens = 0; } else { long durationNanos = currentNanos - previousLastNanos + REMAINDER_NANOS_UPDATER.getAndSet(this, 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index 3bfa8238d86a3..e83f34bfa35c9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -93,19 +93,33 @@ void shouldSupportFractionsWhenUpdatingTokens() { @Test void shouldSupportFractionsAndRetainLeftoverWhenUpdatingTokens() { asyncTokenBucket = - AsyncTokenBucket.builder().capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + AsyncTokenBucket.builder().capacity(100) + .resolutionNanos(TimeUnit.MILLISECONDS.toNanos(1)) + .rate(10) + .initialTokens(0) + .clock(clockSource) + .build(); for (int i = 0; i < 150; i++) { incrementMillis(1); } assertEquals(asyncTokenBucket.getTokens(), 1); incrementMillis(150); assertEquals(asyncTokenBucket.getTokens(), 3); + incrementMillis(1); + assertEquals(asyncTokenBucket.getTokens(), 3); + incrementMillis(99); + assertEquals(asyncTokenBucket.getTokens(), 4); } @Test void shouldSupportFractionsAndRetainLeftoverWhenUpdatingTokens2() { asyncTokenBucket = - AsyncTokenBucket.builder().capacity(100).rate(1).initialTokens(0).clock(clockSource).build(); + AsyncTokenBucket.builder().capacity(100) + .resolutionNanos(TimeUnit.MILLISECONDS.toNanos(1)) + .rate(1) + .initialTokens(0) + .clock(clockSource) + .build(); for (int i = 0; i < 150; i++) { incrementMillis(1); assertEquals(asyncTokenBucket.tokens((i + 1) % 31 == 0), 0); @@ -114,9 +128,9 @@ void shouldSupportFractionsAndRetainLeftoverWhenUpdatingTokens2() { assertEquals(asyncTokenBucket.tokens(true), 0); incrementMillis(699); assertEquals(asyncTokenBucket.tokens(true), 0); - incrementMillis(2); + incrementMillis(1); assertEquals(asyncTokenBucket.tokens(true), 1); - incrementMillis(999); + incrementMillis(1000); assertEquals(asyncTokenBucket.tokens(true), 2); } From 74638912c750e47af769b265ba18d01094ffc503 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 18:29:19 +0200 Subject: [PATCH 35/63] Improve eventual consistency test --- .../apache/pulsar/broker/qos/AsyncTokenBucketTest.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index e83f34bfa35c9..142320ac52804 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -23,7 +23,6 @@ import static org.testng.Assert.assertEquals; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import org.assertj.core.data.Offset; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -216,7 +215,7 @@ void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates @Test void shouldHandleEventualConsistency() { AtomicLong offset = new AtomicLong(0); - long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(100); + long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(1); DefaultMonotonicSnapshotClock monotonicSnapshotClock = new DefaultMonotonicSnapshotClock(resolutionNanos, () -> offset.get() + manualClockSource.get()); @@ -232,8 +231,7 @@ void shouldHandleEventualConsistency() { } assertThat(asyncTokenBucket.tokens(true)) // since the rate is 1/ms and the test increments the clock by 1ms and consumes 1 token in each - // iteration, the tokens should be greater than or equal to the initial tokens - .isGreaterThanOrEqualTo(initialTokens) - .isCloseTo(initialTokens, Offset.offset(1000L)); + // iteration, the tokens should be equal to the initial tokens + .isEqualTo(initialTokens); } } \ No newline at end of file From b04752d01cf32b0a00b3ceae335ba325fc8a22bc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 18:51:43 +0200 Subject: [PATCH 36/63] If condition when newTokens should be 0 --- .../java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index fb35e129a5e72..6ecb71f551d0e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -229,7 +229,7 @@ private long calculateNewTokensSinceLastUpdate(long currentNanos) { return currentLastNanos; } }); - if (previousLastNanos == Long.MIN_VALUE || previousLastNanos >= currentNanos) { + if (previousLastNanos == Long.MIN_VALUE || currentNanos != lastNanos) { newTokens = 0; } else { long durationNanos = currentNanos - previousLastNanos + REMAINDER_NANOS_UPDATER.getAndSet(this, 0); From ea4f7d597a5eafc18ce3ebc0809dc53f4af31d6b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 18:57:35 +0200 Subject: [PATCH 37/63] Improve logic to update lastNanos so that races are prevented with CAS --- .../pulsar/broker/qos/AsyncTokenBucket.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index 6ecb71f551d0e..9a49528333504 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -219,17 +219,19 @@ private boolean shouldUpdateTokensImmediately(long currentNanos, boolean forceUp * @return the number of new tokens to add since the last update */ private long calculateNewTokensSinceLastUpdate(long currentNanos) { + long previousLastNanos = lastNanos; + long newLastNanos; + // update lastNanos only if at least resolutionNanos/2 nanoseconds has passed since the last update + if (currentNanos >= previousLastNanos + resolutionNanos / 2) { + newLastNanos = currentNanos; + } else { + newLastNanos = previousLastNanos; + } long newTokens; - long previousLastNanos = LAST_NANOS_UPDATER.getAndUpdate(this, - currentLastNanos -> { - // update lastNanos only if at least resolutionNanos/2 nanoseconds has passed since the last update - if (currentNanos >= currentLastNanos + resolutionNanos / 2) { - return currentNanos; - } else { - return currentLastNanos; - } - }); - if (previousLastNanos == Long.MIN_VALUE || currentNanos != lastNanos) { + if (newLastNanos == previousLastNanos + // prevent races with a CAS update of lastNanos + || !LAST_NANOS_UPDATER.compareAndSet(this, previousLastNanos, newLastNanos) + || previousLastNanos == Long.MIN_VALUE) { newTokens = 0; } else { long durationNanos = currentNanos - previousLastNanos + REMAINDER_NANOS_UPDATER.getAndSet(this, 0); From ca7cae4df723b68ee32a94068bf707d2713abce8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 20:25:46 +0200 Subject: [PATCH 38/63] Add "getTokensUpdatesTokens" to AsyncTokenBucket to reduce eventual consistency which hinders accuracy - since AsyncTokenBucket is now well optimized for strong consistency, this is a useful change --- .../pulsar/broker/qos/AsyncTokenBucket.java | 18 ++++++++++++------ .../broker/qos/AsyncTokenBucketBuilder.java | 6 ++++++ .../qos/DynamicRateAsyncTokenBucket.java | 6 +++--- .../DynamicRateAsyncTokenBucketBuilder.java | 2 +- .../broker/qos/FinalRateAsyncTokenBucket.java | 5 +++-- .../qos/FinalRateAsyncTokenBucketBuilder.java | 2 +- .../persistent/DispatchRateLimiter.java | 4 ++++ .../persistent/SubscribeRateLimiter.java | 3 ++- 8 files changed, 32 insertions(+), 14 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index 9a49528333504..55211fc1cb738 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -119,10 +119,14 @@ public static void resetToDefaultEventualConsistentTokensView() { */ private final LongAdder pendingConsumedTokens = new LongAdder(); - protected AsyncTokenBucket(MonotonicSnapshotClock clockSource, long resolutionNanos) { + private final boolean getTokensUpdatesTokens; + + protected AsyncTokenBucket(MonotonicSnapshotClock clockSource, long resolutionNanos, + boolean getTokensUpdatesTokens) { this.clockSource = clockSource; this.resolutionNanos = resolutionNanos; this.lastNanos = Long.MIN_VALUE; + this.getTokensUpdatesTokens = getTokensUpdatesTokens; } public static FinalRateAsyncTokenBucketBuilder builder() { @@ -328,10 +332,11 @@ public long calculateThrottlingDuration() { /** * Returns the current number of tokens in the bucket. - * The token balance is updated if the configured resolutionNanos has passed since the last update. + * The token balance is updated if the configured resolutionNanos has passed since the last update unless + * getTokensUpdatesTokens is true. */ public final long getTokens() { - return tokens(false); + return tokens(getTokensUpdatesTokens); } public abstract long getRate(); @@ -339,12 +344,13 @@ public final long getTokens() { /** * Checks if the bucket contains tokens. * The token balance is updated before the comparison if the configured resolutionNanos has passed since the last - * update. It's possible that the returned result is not definite since the token balance is eventually consistent. + * update. It's possible that the returned result is not definite since the token balance is eventually consistent + * if getTokensUpdatesTokens is false. * * @return true if the bucket contains tokens, false otherwise */ public boolean containsTokens() { - return containsTokens(false); + return containsTokens(getTokensUpdatesTokens); } /** @@ -356,7 +362,7 @@ public boolean containsTokens() { * @param forceUpdateTokens if true, the token balance is updated before the comparison * @return true if the bucket contains tokens, false otherwise */ - public boolean containsTokens(boolean forceUpdateTokens) { + protected boolean containsTokens(boolean forceUpdateTokens) { return tokens(forceUpdateTokens) > 0; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java index ee256d5a37d64..d96f4d16932de 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java @@ -23,6 +23,7 @@ public abstract class AsyncTokenBucketBuilder> { protected MonotonicSnapshotClock clock = AsyncTokenBucket.DEFAULT_SNAPSHOT_CLOCK; protected long resolutionNanos = AsyncTokenBucket.defaultResolutionNanos; + protected boolean getTokensUpdatesTokens; protected AsyncTokenBucketBuilder() { } @@ -41,5 +42,10 @@ public SELF resolutionNanos(long resolutionNanos) { return self(); } + public SELF getTokensUpdatesTokens(boolean getTokensUpdatesTokens) { + this.getTokensUpdatesTokens = getTokensUpdatesTokens; + return self(); + } + public abstract AsyncTokenBucket build(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java index 8edc73d1f51e3..11b0faa280286 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java @@ -34,9 +34,9 @@ public class DynamicRateAsyncTokenBucket extends AsyncTokenBucket { protected DynamicRateAsyncTokenBucket(double capacityFactor, LongSupplier rateFunction, MonotonicSnapshotClock clockSource, LongSupplier ratePeriodNanosFunction, - long resolutionNanos, double initialTokensFactor, - double targetFillFactorAfterThrottling) { - super(clockSource, resolutionNanos); + long resolutionNanos, boolean getTokensUpdatesTokens, + double initialTokensFactor, double targetFillFactorAfterThrottling) { + super(clockSource, resolutionNanos, getTokensUpdatesTokens); this.capacityFactor = capacityFactor; this.rateFunction = rateFunction; this.ratePeriodNanosFunction = ratePeriodNanosFunction; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java index 22270484c72f0..ac20be89a900b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java @@ -65,7 +65,7 @@ public DynamicRateAsyncTokenBucketBuilder targetFillFactorAfterThrottling( public AsyncTokenBucket build() { return new DynamicRateAsyncTokenBucket(this.capacityFactor, this.rateFunction, this.clock, - this.ratePeriodNanosFunction, this.resolutionNanos, + this.ratePeriodNanosFunction, this.resolutionNanos, this.getTokensUpdatesTokens, this.initialFillFactor, targetFillFactorAfterThrottling); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java index 627c5ee1334b2..869f752e4b034 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java @@ -30,8 +30,9 @@ class FinalRateAsyncTokenBucket extends AsyncTokenBucket { private final long targetAmountOfTokensAfterThrottling; protected FinalRateAsyncTokenBucket(long capacity, long rate, MonotonicSnapshotClock clockSource, - long ratePeriodNanos, long resolutionNanos, long initialTokens) { - super(clockSource, resolutionNanos); + long ratePeriodNanos, long resolutionNanos, boolean getTokensUpdatesTokens, + long initialTokens) { + super(clockSource, resolutionNanos, getTokensUpdatesTokens); this.capacity = capacity; this.rate = rate; this.ratePeriodNanos = ratePeriodNanos != -1 ? ratePeriodNanos : ONE_SECOND_NANOS; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java index ff4ed53c6c7fa..b99ad008ea3b7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java @@ -55,7 +55,7 @@ public FinalRateAsyncTokenBucketBuilder initialTokens(long initialTokens) { public AsyncTokenBucket build() { return new FinalRateAsyncTokenBucket(this.capacity != null ? this.capacity : this.rate, this.rate, this.clock, - this.ratePeriodNanos, this.resolutionNanos, + this.ratePeriodNanos, this.resolutionNanos, this.getTokensUpdatesTokens, this.initialTokens != null ? this.initialTokens : this.rate ); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java index 0f4323e5fef3d..0073e93efda13 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java @@ -224,6 +224,7 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnMessage = AsyncTokenBucket.builderForDynamicRate() + .getTokensUpdatesTokens(true) .clock(clock) .rateFunction(() -> getRelativeDispatchRateInMsg(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) @@ -231,6 +232,7 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { } else { this.dispatchRateLimiterOnMessage = AsyncTokenBucket.builder() + .getTokensUpdatesTokens(true) .clock(clock) .rate(msgRate).ratePeriodNanos(ratePeriodNanos) .build(); @@ -244,6 +246,7 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnByte = AsyncTokenBucket.builderForDynamicRate() + .getTokensUpdatesTokens(true) .clock(clock) .rateFunction(() -> getRelativeDispatchRateInByte(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) @@ -251,6 +254,7 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { } else { this.dispatchRateLimiterOnByte = AsyncTokenBucket.builder() + .getTokensUpdatesTokens(true) .clock(clock) .rate(byteRate).ratePeriodNanos(ratePeriodNanos) .build(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java index e9951b83d7947..338a71680e411 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java @@ -70,7 +70,7 @@ public synchronized boolean tryAcquire(ConsumerIdentifier consumerIdentifier) { if (tokenBucket == null) { return true; } - if (!tokenBucket.containsTokens(true)) { + if (!tokenBucket.containsTokens()) { return false; } tokenBucket.consumeTokens(1); @@ -118,6 +118,7 @@ private synchronized void updateSubscribeRate(ConsumerIdentifier consumerIdentif if (ratePerConsumer > 0) { AsyncTokenBucket tokenBucket = AsyncTokenBucket.builder() + .getTokensUpdatesTokens(true) .clock(brokerService.getPulsar().getMonotonicSnapshotClock()) .rate(ratePerConsumer).ratePeriodNanos(ratePeriodNanos).build(); this.subscribeRateLimiter.put(consumerIdentifier, tokenBucket); From 543c9785bfdd6008f7f0199474675918f8a8c731 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 20:35:19 +0200 Subject: [PATCH 39/63] Reduce test flakiness --- ...criptionMessageDispatchThrottlingTest.java | 56 +++++++++++++++---- 1 file changed, 46 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index ce554ab2d9c00..9d91c9adc3665 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -18,11 +18,14 @@ */ package org.apache.pulsar.client.api; +import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; import com.google.common.collect.Sets; +import java.time.Duration; import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; @@ -272,10 +275,21 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce Assert.fail("Should only have PersistentDispatcher in this test"); } final DispatchRateLimiter subDispatchRateLimiter = subRateLimiter; - Awaitility.await().untilAsserted(() -> { + MutableBoolean brokerConfigRetried = new MutableBoolean(false); + Awaitility.await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); - Assert.assertTrue(brokerDispatchRateLimiter != null - && brokerDispatchRateLimiter.getDispatchRateOnByte() > 0); + try { + assertThat(brokerDispatchRateLimiter) + .isNotNull() + .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(brokerRate)); + } catch (AssertionError e) { + if (!brokerConfigRetried.booleanValue()) { + brokerConfigRetried.setTrue(); + admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + brokerRate); + Thread.sleep(2000L); + } + throw e; + } DispatchRateLimiter topicDispatchRateLimiter = topic.getDispatchRateLimiter().orElse(null); Assert.assertTrue(topicDispatchRateLimiter != null && topicDispatchRateLimiter.getDispatchRateOnByte() > 0); @@ -450,10 +464,21 @@ private void testDispatchRate(SubscriptionType subscription, Assert.fail("Should only have PersistentDispatcher in this test"); } final DispatchRateLimiter subDispatchRateLimiter = subRateLimiter; - Awaitility.await().untilAsserted(() -> { + MutableBoolean brokerConfigRetried = new MutableBoolean(false); + Awaitility.await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); - Assert.assertTrue(brokerDispatchRateLimiter != null - && brokerDispatchRateLimiter.getDispatchRateOnByte() > 0); + try { + assertThat(brokerDispatchRateLimiter) + .isNotNull() + .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(brokerRate)); + } catch (AssertionError e) { + if (!brokerConfigRetried.booleanValue()) { + brokerConfigRetried.setTrue(); + admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + brokerRate); + Thread.sleep(2000L); + } + throw e; + } DispatchRateLimiter topicDispatchRateLimiter = topic.getDispatchRateLimiter().orElse(null); Assert.assertTrue(topicDispatchRateLimiter != null && topicDispatchRateLimiter.getDispatchRateOnByte() > 0); @@ -576,10 +601,21 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri Producer producer1 = pulsarClient.newProducer().topic(topicName1).create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2).create(); - Awaitility.await().untilAsserted(() -> { - DispatchRateLimiter rateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); - Assert.assertTrue(rateLimiter != null - && rateLimiter.getDispatchRateOnByte() > 0); + MutableBoolean brokerConfigRetried = new MutableBoolean(false); + Awaitility.await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { + DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); + try { + assertThat(brokerDispatchRateLimiter) + .isNotNull() + .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(byteRate)); + } catch (AssertionError e) { + if (!brokerConfigRetried.booleanValue()) { + brokerConfigRetried.setTrue(); + admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + byteRate); + Thread.sleep(2000L); + } + throw e; + } }); long start = System.currentTimeMillis(); From d1591980bb8d63e3e87eb40d0c7c4eeab4169d5d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 21:50:42 +0200 Subject: [PATCH 40/63] Let MessageDispatchThrottlingTest#reset handle deletion --- .../api/SubscriptionMessageDispatchThrottlingTest.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index 9d91c9adc3665..da323f1dbab43 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -316,9 +316,6 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce producer.close(); admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(initBytes)); - - admin.topics().delete(topicName, true); - admin.namespaces().deleteNamespace(namespace); } /** @@ -508,8 +505,6 @@ private void testDispatchRate(SubscriptionType subscription, consumer.close(); producer.close(); admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(initBytes)); - admin.topics().delete(topicName, true); - admin.namespaces().deleteNamespace(namespace); } /** From 936cdb0fba5f9dde260a2ca3f18a352f9ca2346e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 21:58:09 +0200 Subject: [PATCH 41/63] Reduce test flakiness for waiting to new rate to be applied --- ...criptionMessageDispatchThrottlingTest.java | 76 ++++++------------- 1 file changed, 25 insertions(+), 51 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index da323f1dbab43..958a9b6a073bf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -25,7 +25,6 @@ import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; @@ -33,6 +32,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.policies.data.DispatchRate; import org.awaitility.Awaitility; import org.slf4j.Logger; @@ -244,7 +244,7 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce admin.namespaces().setSubscriptionDispatchRate(namespace, subscriptionDispatchRate); admin.namespaces().setDispatchRate(namespace, topicDispatchRate); long initBytes = pulsar.getConfiguration().getDispatchThrottlingRatePerTopicInByte(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + brokerRate); + updateBrokerDispatchThrottlingRateInBytes(brokerRate); final int numProducedMessages = 30; final CountDownLatch latch = new CountDownLatch(numProducedMessages); @@ -275,21 +275,11 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce Assert.fail("Should only have PersistentDispatcher in this test"); } final DispatchRateLimiter subDispatchRateLimiter = subRateLimiter; - MutableBoolean brokerConfigRetried = new MutableBoolean(false); Awaitility.await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); - try { - assertThat(brokerDispatchRateLimiter) - .isNotNull() - .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(brokerRate)); - } catch (AssertionError e) { - if (!brokerConfigRetried.booleanValue()) { - brokerConfigRetried.setTrue(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + brokerRate); - Thread.sleep(2000L); - } - throw e; - } + assertThat(brokerDispatchRateLimiter) + .isNotNull() + .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(brokerRate)); DispatchRateLimiter topicDispatchRateLimiter = topic.getDispatchRateLimiter().orElse(null); Assert.assertTrue(topicDispatchRateLimiter != null && topicDispatchRateLimiter.getDispatchRateOnByte() > 0); @@ -315,7 +305,7 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce consumer.close(); producer.close(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(initBytes)); + updateBrokerDispatchThrottlingRateInBytes(initBytes); } /** @@ -412,7 +402,7 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT private void testDispatchRate(SubscriptionType subscription, int brokerRate, int topicRate, int subRate, int expectRate) throws Exception { - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/throttlingAll"); final String subName = "my-subscriber-name-" + subscription; @@ -430,7 +420,7 @@ private void testDispatchRate(SubscriptionType subscription, admin.namespaces().setSubscriptionDispatchRate(namespace, subscriptionDispatchRate); admin.namespaces().setDispatchRate(namespace, topicDispatchRate); long initBytes = pulsar.getConfiguration().getDispatchThrottlingRatePerTopicInByte(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + brokerRate); + updateBrokerDispatchThrottlingRateInBytes(brokerRate); final int numProducedMessages = 30; final CountDownLatch latch = new CountDownLatch(numProducedMessages); @@ -461,21 +451,11 @@ private void testDispatchRate(SubscriptionType subscription, Assert.fail("Should only have PersistentDispatcher in this test"); } final DispatchRateLimiter subDispatchRateLimiter = subRateLimiter; - MutableBoolean brokerConfigRetried = new MutableBoolean(false); Awaitility.await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); - try { - assertThat(brokerDispatchRateLimiter) - .isNotNull() - .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(brokerRate)); - } catch (AssertionError e) { - if (!brokerConfigRetried.booleanValue()) { - brokerConfigRetried.setTrue(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + brokerRate); - Thread.sleep(2000L); - } - throw e; - } + assertThat(brokerDispatchRateLimiter) + .isNotNull() + .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(brokerRate)); DispatchRateLimiter topicDispatchRateLimiter = topic.getDispatchRateLimiter().orElse(null); Assert.assertTrue(topicDispatchRateLimiter != null && topicDispatchRateLimiter.getDispatchRateOnByte() > 0); @@ -504,7 +484,18 @@ private void testDispatchRate(SubscriptionType subscription, consumer.close(); producer.close(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(initBytes)); + updateBrokerDispatchThrottlingRateInBytes(initBytes); + } + + private void updateBrokerDispatchThrottlingRateInBytes(long bytes) throws PulsarAdminException { + admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(bytes)); + long expectedBytes = bytes > 0L ? bytes : -1L; + await().untilAsserted(() -> { + DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); + assertThat(brokerDispatchRateLimiter) + .isNotNull() + .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(expectedBytes)); + }); } /** @@ -557,7 +548,7 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri long initBytes = pulsar.getConfiguration().getDispatchThrottlingRatePerTopicInByte(); final int byteRate = 1000; - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + byteRate); + updateBrokerDispatchThrottlingRateInBytes(byteRate); Awaitility.await().untilAsserted(() -> { Assert.assertEquals(pulsar.getConfiguration().getDispatchThrottlingRateInByte(), byteRate); @@ -596,23 +587,6 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri Producer producer1 = pulsarClient.newProducer().topic(topicName1).create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2).create(); - MutableBoolean brokerConfigRetried = new MutableBoolean(false); - Awaitility.await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { - DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); - try { - assertThat(brokerDispatchRateLimiter) - .isNotNull() - .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(byteRate)); - } catch (AssertionError e) { - if (!brokerConfigRetried.booleanValue()) { - brokerConfigRetried.setTrue(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + byteRate); - Thread.sleep(2000L); - } - throw e; - } - }); - long start = System.currentTimeMillis(); // Asynchronously produce messages for (int i = 0; i < numProducedMessagesEachTopic; i++) { @@ -631,7 +605,7 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri consumer2.close(); producer1.close(); producer2.close(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(initBytes)); + updateBrokerDispatchThrottlingRateInBytes(initBytes); log.info("-- Exiting {} test --", methodName); } From dae82208f60142129e5f4eff7f59eaffd0895b21 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 22:11:01 +0200 Subject: [PATCH 42/63] Prevent NPEs in DispatchRateLimiter when limit has changed --- .../persistent/DispatchRateLimiter.java | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java index 0073e93efda13..b602e0edd8642 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java @@ -77,7 +77,9 @@ public DispatchRateLimiter(BrokerService brokerService) { * @return */ public long getAvailableDispatchRateLimitOnMsg() { - return dispatchRateLimiterOnMessage == null ? -1 : Math.max(dispatchRateLimiterOnMessage.getTokens(), 0); + AsyncTokenBucket localDispatchRateLimiterOnMessage = dispatchRateLimiterOnMessage; + return localDispatchRateLimiterOnMessage == null ? -1 : + Math.max(localDispatchRateLimiterOnMessage.getTokens(), 0); } /** @@ -86,7 +88,8 @@ public long getAvailableDispatchRateLimitOnMsg() { * @return */ public long getAvailableDispatchRateLimitOnByte() { - return dispatchRateLimiterOnByte == null ? -1 : Math.max(dispatchRateLimiterOnByte.getTokens(), 0); + AsyncTokenBucket localDispatchRateLimiterOnByte = dispatchRateLimiterOnByte; + return localDispatchRateLimiterOnByte == null ? -1 : Math.max(localDispatchRateLimiterOnByte.getTokens(), 0); } /** @@ -96,11 +99,13 @@ public long getAvailableDispatchRateLimitOnByte() { * @param byteSize */ public void consumeDispatchQuota(long numberOfMessages, long byteSize) { - if (numberOfMessages > 0 && dispatchRateLimiterOnMessage != null) { - dispatchRateLimiterOnMessage.consumeTokens(numberOfMessages); + AsyncTokenBucket localDispatchRateLimiterOnMessage = dispatchRateLimiterOnMessage; + if (numberOfMessages > 0 && localDispatchRateLimiterOnMessage != null) { + localDispatchRateLimiterOnMessage.consumeTokens(numberOfMessages); } - if (byteSize > 0 && dispatchRateLimiterOnByte != null) { - dispatchRateLimiterOnByte.consumeTokens(byteSize); + AsyncTokenBucket localDispatchRateLimiterOnByte = dispatchRateLimiterOnByte; + if (byteSize > 0 && localDispatchRateLimiterOnByte != null) { + localDispatchRateLimiterOnByte.consumeTokens(byteSize); } } @@ -282,7 +287,8 @@ private long getRelativeDispatchRateInByte(DispatchRate dispatchRate) { * @return */ public long getDispatchRateOnMsg() { - return dispatchRateLimiterOnMessage != null ? dispatchRateLimiterOnMessage.getRate() : -1; + AsyncTokenBucket localDispatchRateLimiterOnMessage = dispatchRateLimiterOnMessage; + return localDispatchRateLimiterOnMessage != null ? localDispatchRateLimiterOnMessage.getRate() : -1; } /** @@ -291,7 +297,8 @@ public long getDispatchRateOnMsg() { * @return */ public long getDispatchRateOnByte() { - return dispatchRateLimiterOnByte != null ? dispatchRateLimiterOnByte.getRate() : -1; + AsyncTokenBucket localDispatchRateLimiterOnByte = dispatchRateLimiterOnByte; + return localDispatchRateLimiterOnByte != null ? localDispatchRateLimiterOnByte.getRate() : -1; } From d62f97ea3a8009c85bd894e849195eb58ab0412f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 22:28:02 +0200 Subject: [PATCH 43/63] Fix switchToConsistentTokensView behavior --- .../java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index 55211fc1cb738..08f472119fde5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -166,7 +166,8 @@ private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolea // calculate the number of new tokens since the last update newTokens = calculateNewTokensSinceLastUpdate(currentNanos); } - if (newTokens > 0) { + // update tokens if there are new tokens or if resolutionNanos is set to 0 which is currently used for testing + if (newTokens > 0 || resolutionNanos == 0) { // flush the pendingConsumedTokens by calling "sumThenReset" long currentPendingConsumedTokens = pendingConsumedTokens.sumThenReset(); // calculate the token delta by subtracting the consumed tokens from the new tokens From 1d685229261c47cd4d57d5e2eb848c292fa4e414 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 22:28:45 +0200 Subject: [PATCH 44/63] Revert using getTokensUpdatesTokens mode by default since eventual consistency should be fine --- .../java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java | 2 +- .../pulsar/broker/service/persistent/DispatchRateLimiter.java | 4 ---- .../broker/service/persistent/SubscribeRateLimiter.java | 3 +-- 3 files changed, 2 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index 08f472119fde5..33a79648d0f98 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -363,7 +363,7 @@ public boolean containsTokens() { * @param forceUpdateTokens if true, the token balance is updated before the comparison * @return true if the bucket contains tokens, false otherwise */ - protected boolean containsTokens(boolean forceUpdateTokens) { + public boolean containsTokens(boolean forceUpdateTokens) { return tokens(forceUpdateTokens) > 0; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java index b602e0edd8642..5df6204f6b0bb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java @@ -229,7 +229,6 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnMessage = AsyncTokenBucket.builderForDynamicRate() - .getTokensUpdatesTokens(true) .clock(clock) .rateFunction(() -> getRelativeDispatchRateInMsg(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) @@ -237,7 +236,6 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { } else { this.dispatchRateLimiterOnMessage = AsyncTokenBucket.builder() - .getTokensUpdatesTokens(true) .clock(clock) .rate(msgRate).ratePeriodNanos(ratePeriodNanos) .build(); @@ -251,7 +249,6 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnByte = AsyncTokenBucket.builderForDynamicRate() - .getTokensUpdatesTokens(true) .clock(clock) .rateFunction(() -> getRelativeDispatchRateInByte(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) @@ -259,7 +256,6 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { } else { this.dispatchRateLimiterOnByte = AsyncTokenBucket.builder() - .getTokensUpdatesTokens(true) .clock(clock) .rate(byteRate).ratePeriodNanos(ratePeriodNanos) .build(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java index 338a71680e411..e9951b83d7947 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java @@ -70,7 +70,7 @@ public synchronized boolean tryAcquire(ConsumerIdentifier consumerIdentifier) { if (tokenBucket == null) { return true; } - if (!tokenBucket.containsTokens()) { + if (!tokenBucket.containsTokens(true)) { return false; } tokenBucket.consumeTokens(1); @@ -118,7 +118,6 @@ private synchronized void updateSubscribeRate(ConsumerIdentifier consumerIdentif if (ratePerConsumer > 0) { AsyncTokenBucket tokenBucket = AsyncTokenBucket.builder() - .getTokensUpdatesTokens(true) .clock(brokerService.getPulsar().getMonotonicSnapshotClock()) .rate(ratePerConsumer).ratePeriodNanos(ratePeriodNanos).build(); this.subscribeRateLimiter.put(consumerIdentifier, tokenBucket); From 38e9f7b23f3a93bdf6425fe1517881921916dfdc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 22:33:59 +0200 Subject: [PATCH 45/63] Rename getTokensUpdatesTokens to consistentTokensView --- .../apache/pulsar/broker/qos/AsyncTokenBucket.java | 14 +++++++------- .../pulsar/broker/qos/AsyncTokenBucketBuilder.java | 6 +++--- .../broker/qos/DynamicRateAsyncTokenBucket.java | 4 ++-- .../qos/DynamicRateAsyncTokenBucketBuilder.java | 2 +- .../broker/qos/FinalRateAsyncTokenBucket.java | 4 ++-- .../qos/FinalRateAsyncTokenBucketBuilder.java | 2 +- 6 files changed, 16 insertions(+), 16 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index 33a79648d0f98..96efcb6a215f3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -119,14 +119,14 @@ public static void resetToDefaultEventualConsistentTokensView() { */ private final LongAdder pendingConsumedTokens = new LongAdder(); - private final boolean getTokensUpdatesTokens; + private final boolean consistentTokensView; protected AsyncTokenBucket(MonotonicSnapshotClock clockSource, long resolutionNanos, - boolean getTokensUpdatesTokens) { + boolean consistentTokensView) { this.clockSource = clockSource; this.resolutionNanos = resolutionNanos; this.lastNanos = Long.MIN_VALUE; - this.getTokensUpdatesTokens = getTokensUpdatesTokens; + this.consistentTokensView = consistentTokensView; } public static FinalRateAsyncTokenBucketBuilder builder() { @@ -334,10 +334,10 @@ public long calculateThrottlingDuration() { /** * Returns the current number of tokens in the bucket. * The token balance is updated if the configured resolutionNanos has passed since the last update unless - * getTokensUpdatesTokens is true. + * consistentTokensView is true. */ public final long getTokens() { - return tokens(getTokensUpdatesTokens); + return tokens(consistentTokensView); } public abstract long getRate(); @@ -346,12 +346,12 @@ public final long getTokens() { * Checks if the bucket contains tokens. * The token balance is updated before the comparison if the configured resolutionNanos has passed since the last * update. It's possible that the returned result is not definite since the token balance is eventually consistent - * if getTokensUpdatesTokens is false. + * if consistentTokensView is false. * * @return true if the bucket contains tokens, false otherwise */ public boolean containsTokens() { - return containsTokens(getTokensUpdatesTokens); + return containsTokens(consistentTokensView); } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java index d96f4d16932de..cad90561a13a5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java @@ -23,7 +23,7 @@ public abstract class AsyncTokenBucketBuilder> { protected MonotonicSnapshotClock clock = AsyncTokenBucket.DEFAULT_SNAPSHOT_CLOCK; protected long resolutionNanos = AsyncTokenBucket.defaultResolutionNanos; - protected boolean getTokensUpdatesTokens; + protected boolean consistentTokensView; protected AsyncTokenBucketBuilder() { } @@ -42,8 +42,8 @@ public SELF resolutionNanos(long resolutionNanos) { return self(); } - public SELF getTokensUpdatesTokens(boolean getTokensUpdatesTokens) { - this.getTokensUpdatesTokens = getTokensUpdatesTokens; + public SELF consistentTokensView(boolean consistentTokensView) { + this.consistentTokensView = consistentTokensView; return self(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java index 11b0faa280286..05ac138d0289a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java @@ -34,9 +34,9 @@ public class DynamicRateAsyncTokenBucket extends AsyncTokenBucket { protected DynamicRateAsyncTokenBucket(double capacityFactor, LongSupplier rateFunction, MonotonicSnapshotClock clockSource, LongSupplier ratePeriodNanosFunction, - long resolutionNanos, boolean getTokensUpdatesTokens, + long resolutionNanos, boolean consistentTokensView, double initialTokensFactor, double targetFillFactorAfterThrottling) { - super(clockSource, resolutionNanos, getTokensUpdatesTokens); + super(clockSource, resolutionNanos, consistentTokensView); this.capacityFactor = capacityFactor; this.rateFunction = rateFunction; this.ratePeriodNanosFunction = ratePeriodNanosFunction; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java index ac20be89a900b..ce38ff2a4078c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java @@ -65,7 +65,7 @@ public DynamicRateAsyncTokenBucketBuilder targetFillFactorAfterThrottling( public AsyncTokenBucket build() { return new DynamicRateAsyncTokenBucket(this.capacityFactor, this.rateFunction, this.clock, - this.ratePeriodNanosFunction, this.resolutionNanos, this.getTokensUpdatesTokens, + this.ratePeriodNanosFunction, this.resolutionNanos, this.consistentTokensView, this.initialFillFactor, targetFillFactorAfterThrottling); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java index 869f752e4b034..3f5edcbf4f06d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java @@ -30,9 +30,9 @@ class FinalRateAsyncTokenBucket extends AsyncTokenBucket { private final long targetAmountOfTokensAfterThrottling; protected FinalRateAsyncTokenBucket(long capacity, long rate, MonotonicSnapshotClock clockSource, - long ratePeriodNanos, long resolutionNanos, boolean getTokensUpdatesTokens, + long ratePeriodNanos, long resolutionNanos, boolean consistentTokensView, long initialTokens) { - super(clockSource, resolutionNanos, getTokensUpdatesTokens); + super(clockSource, resolutionNanos, consistentTokensView); this.capacity = capacity; this.rate = rate; this.ratePeriodNanos = ratePeriodNanos != -1 ? ratePeriodNanos : ONE_SECOND_NANOS; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java index b99ad008ea3b7..848eaaaca0b49 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java @@ -55,7 +55,7 @@ public FinalRateAsyncTokenBucketBuilder initialTokens(long initialTokens) { public AsyncTokenBucket build() { return new FinalRateAsyncTokenBucket(this.capacity != null ? this.capacity : this.rate, this.rate, this.clock, - this.ratePeriodNanos, this.resolutionNanos, this.getTokensUpdatesTokens, + this.ratePeriodNanos, this.resolutionNanos, this.consistentTokensView, this.initialTokens != null ? this.initialTokens : this.rate ); } From f5c3e5714ec50407d429602135893165d789f09c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 22:37:19 +0200 Subject: [PATCH 46/63] Use consistent tokens view for SubscribeRateLimiter --- .../java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java | 2 +- .../pulsar/broker/service/persistent/SubscribeRateLimiter.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index 96efcb6a215f3..4f9e35ca3d8d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -363,7 +363,7 @@ public boolean containsTokens() { * @param forceUpdateTokens if true, the token balance is updated before the comparison * @return true if the bucket contains tokens, false otherwise */ - public boolean containsTokens(boolean forceUpdateTokens) { + protected boolean containsTokens(boolean forceUpdateTokens) { return tokens(forceUpdateTokens) > 0; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java index e9951b83d7947..c465a36359bd2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java @@ -70,7 +70,7 @@ public synchronized boolean tryAcquire(ConsumerIdentifier consumerIdentifier) { if (tokenBucket == null) { return true; } - if (!tokenBucket.containsTokens(true)) { + if (!tokenBucket.containsTokens()) { return false; } tokenBucket.consumeTokens(1); @@ -118,6 +118,7 @@ private synchronized void updateSubscribeRate(ConsumerIdentifier consumerIdentif if (ratePerConsumer > 0) { AsyncTokenBucket tokenBucket = AsyncTokenBucket.builder() + .consistentTokensView(true) .clock(brokerService.getPulsar().getMonotonicSnapshotClock()) .rate(ratePerConsumer).ratePeriodNanos(ratePeriodNanos).build(); this.subscribeRateLimiter.put(consumerIdentifier, tokenBucket); From b79ed859b5f87b4ab4207072c34c92869f60e637 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 23:04:42 +0200 Subject: [PATCH 47/63] Fix issue with restartBroker in tests --- .../auth/MockedPulsarServiceBaseTest.java | 34 +++++++++++-------- 1 file changed, 20 insertions(+), 14 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index 8dd2fc1c3c26d..e46b76205ecf0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -170,20 +170,24 @@ protected final void resetConfig() { protected final void internalSetup() throws Exception { init(); - lookupUrl = new URI(brokerUrl.toString()); - if (isTcpLookup) { - lookupUrl = new URI(pulsar.getBrokerServiceUrl()); - + lookupUrl = resolveLookupUrl(); + if (isTcpLookup && enableBrokerGateway) { // setup port forwarding from the advertised port to the listen port - if (enableBrokerGateway) { - InetSocketAddress gatewayAddress = new InetSocketAddress(lookupUrl.getHost(), lookupUrl.getPort()); - InetSocketAddress brokerAddress = new InetSocketAddress("127.0.0.1", pulsar.getBrokerListenPort().get()); - brokerGateway = new PortForwarder(gatewayAddress, brokerAddress); - } + InetSocketAddress gatewayAddress = new InetSocketAddress(lookupUrl.getHost(), lookupUrl.getPort()); + InetSocketAddress brokerAddress = new InetSocketAddress("127.0.0.1", pulsar.getBrokerListenPort().get()); + brokerGateway = new PortForwarder(gatewayAddress, brokerAddress); } pulsarClient = newPulsarClient(lookupUrl.toString(), 0); } + private URI resolveLookupUrl() { + if (isTcpLookup) { + return URI.create(pulsar.getBrokerServiceUrl()); + } else { + return URI.create(brokerUrl.toString()); + } + } + protected final void internalSetup(ServiceConfiguration serviceConfiguration) throws Exception { this.conf = serviceConfiguration; internalSetup(); @@ -384,12 +388,14 @@ protected void startBroker() throws Exception { brokerUrl = pulsar.getWebServiceAddress() != null ? new URL(pulsar.getWebServiceAddress()) : null; brokerUrlTls = pulsar.getWebServiceAddressTls() != null ? new URL(pulsar.getWebServiceAddressTls()) : null; - if (admin != null) { - admin.close(); - if (MockUtil.isMock(admin)) { - Mockito.reset(admin); - } + URI newLookupUrl = resolveLookupUrl(); + if (pulsarClient != null && (lookupUrl == null || !newLookupUrl.equals(lookupUrl))) { + pulsarClient.shutdown(); + lookupUrl = newLookupUrl; + pulsarClient = newPulsarClient(lookupUrl.toString(), 0); } + + closeAdmin(); PulsarAdminBuilder pulsarAdminBuilder = PulsarAdmin.builder().serviceHttpUrl(brokerUrl != null ? brokerUrl.toString() : brokerUrlTls.toString()); From 5910b3c714d8097033247a626f1422b0f17d58b7 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 23:16:28 +0200 Subject: [PATCH 48/63] Ignore metadata change when broker isn't running --- .../org/apache/pulsar/broker/service/BrokerService.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 79e6fb2b02e31..3e9c6302ceade 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 @@ -2506,6 +2506,12 @@ public long getNumberOfNamespaceBundles() { private void handleMetadataChanges(Notification n) { + if (!pulsar.isRunning()) { + // Ignore metadata changes when broker is not running + log.info("Ignoring metadata change since broker is not running (id={}, state={}) {}", pulsar.getBrokerId(), + pulsar.getState(), n); + return; + } if (n.getType() == NotificationType.Modified && NamespaceResources.pathIsFromNamespace(n.getPath())) { NamespaceName ns = NamespaceResources.namespaceFromPath(n.getPath()); handlePoliciesUpdates(ns); From fda233711eeae58c0536d2b2ec6cc74aa5f2b552 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 23:25:25 +0200 Subject: [PATCH 49/63] Move dispatch throttling tests to broker-api group --- .../apache/pulsar/client/api/MessageDispatchThrottlingTest.java | 2 +- .../client/api/SubscriptionMessageDispatchThrottlingTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index a544c7e13bc83..7b432021b1097 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -63,7 +63,7 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -@Test(groups = "flaky") +@Test(groups = "broker-api") public class MessageDispatchThrottlingTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(MessageDispatchThrottlingTest.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index 958a9b6a073bf..88a680fe33007 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -40,7 +40,7 @@ import org.testng.Assert; import org.testng.annotations.Test; -@Test(groups = "flaky") +@Test(groups = "broker-api") public class SubscriptionMessageDispatchThrottlingTest extends MessageDispatchThrottlingTest { private static final Logger log = LoggerFactory.getLogger(SubscriptionMessageDispatchThrottlingTest.class); From 759fafeae62a9d3956bf90c3e748462db38778d6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 23:35:52 +0200 Subject: [PATCH 50/63] Use AssertJ for better error message --- .../pulsar/client/api/MessageDispatchThrottlingTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index 7b432021b1097..ac78ff7462caa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.spy; @@ -43,6 +44,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.cache.PendingReadsManager; import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -52,7 +54,7 @@ import org.apache.pulsar.common.policies.data.PublishRate; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; -import org.apache.pulsar.broker.qos.AsyncTokenBucket; +import org.assertj.core.data.Offset; import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -585,7 +587,7 @@ public void testRateLimitingMultipleConsumers() throws Exception { Thread.sleep(1000); // rate limiter should have limited messages with at least 10% accuracy (or 2 messages if messageRate is low) - Assert.assertEquals(totalReceived.get(), messageRate, Math.max(messageRate / 10, 2)); + assertThat(totalReceived.get()).isCloseTo(messageRate, Offset.offset(Math.max(messageRate / 10, 2))); consumer1.close(); consumer2.close(); From 5e0a327dea458153501ea99c4e1b3e248e9e3b1a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 23:39:17 +0200 Subject: [PATCH 51/63] Improve test cleanup for retries --- .../api/MessageDispatchThrottlingTest.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index ac78ff7462caa..f1d5fdd36cdb3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -542,6 +542,7 @@ public void testRateLimitingMultipleConsumers() throws Exception { admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); // create producer and topic + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); @@ -568,10 +569,15 @@ public void testRateLimitingMultipleConsumers() throws Exception { throw new RuntimeException(e); } }); + @Cleanup Consumer consumer1 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer2 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer3 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer4 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer5 = consumerBuilder.subscribe(); // deactive cursors @@ -589,12 +595,6 @@ public void testRateLimitingMultipleConsumers() throws Exception { // rate limiter should have limited messages with at least 10% accuracy (or 2 messages if messageRate is low) assertThat(totalReceived.get()).isCloseTo(messageRate, Offset.offset(Math.max(messageRate / 10, 2))); - consumer1.close(); - consumer2.close(); - consumer3.close(); - consumer4.close(); - consumer5.close(); - producer.close(); log.info("-- Exiting {} test --", methodName); } @@ -616,6 +616,7 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { final int messagesPerBatch = 100; final int numProducedMessages = messageRate * messagesPerBatch; // create producer and topic + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).enableBatching(true) .batchingMaxPublishDelay(1, TimeUnit.SECONDS).batchingMaxMessages(messagesPerBatch).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); @@ -636,10 +637,15 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { log.debug("Received message [{}] in the listener", receivedMessage); totalReceived.incrementAndGet(); }); + @Cleanup Consumer consumer1 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer2 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer3 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer4 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer5 = consumerBuilder.subscribe(); // deactive cursors @@ -659,12 +665,6 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { // consumer should not have received all published message due to message-rate throttling Assert.assertEquals(totalReceived.get(), numProducedMessages); - consumer1.close(); - consumer2.close(); - consumer3.close(); - consumer4.close(); - consumer5.close(); - producer.close(); log.info("-- Exiting {} test --", methodName); } @@ -690,12 +690,14 @@ public void testClusterRateLimitingConfiguration(SubscriptionType subscription) admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); // create producer and topic + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); int numMessages = 500; final AtomicInteger totalReceived = new AtomicInteger(0); + @Cleanup Consumer consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-subscriber-name") .subscriptionType(subscription).messageListener((c1, msg) -> { Assert.assertNotNull(msg, "Message cannot be null"); @@ -718,8 +720,6 @@ public void testClusterRateLimitingConfiguration(SubscriptionType subscription) // consumer should not have received all published message due to message-rate throttling Assert.assertNotEquals(totalReceived.get(), numMessages); - consumer.close(); - producer.close(); admin.brokers().updateDynamicConfiguration("dispatchThrottlingRatePerTopicInMsg", Integer.toString(initValue)); log.info("-- Exiting {} test --", methodName); From f024203b229f4fe10ef3a6cd4e4987df082b2078 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 23:42:41 +0200 Subject: [PATCH 52/63] Use unique namespaces --- .../api/MessageDispatchThrottlingTest.java | 33 ++++++++++--------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index f1d5fdd36cdb3..6e0b38bbb24d6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -44,6 +44,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.cache.PendingReadsManager; import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; @@ -152,7 +153,7 @@ public void testMessageRateDynamicallyChange() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); @@ -222,7 +223,7 @@ public void testMessageRateDynamicallyChange() throws Exception { @SuppressWarnings("deprecation") @Test public void testSystemTopicDeliveryNonBlock() throws Exception { - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); final String topicName = "persistent://" + namespace + "/" + UUID.randomUUID().toString().replaceAll("-", ""); admin.topics().createNonPartitionedTopic(topicName); @@ -266,7 +267,7 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr DispatchRateType dispatchRateType) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 100; @@ -334,7 +335,7 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr public void testClusterMsgByteRateLimitingClusterConfig() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 5; final long byteRate = 1024 * 1024;// 1MB rate enough to let all msg to be delivered @@ -409,7 +410,7 @@ public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(Subscriptio throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingAll"; final int messageRate = 10; @@ -477,7 +478,7 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT conf.setDispatchThrottlingOnNonBacklogConsumerEnabled(true); log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingAll"; final String subscriptionName = "my-subscriber-name"; @@ -530,7 +531,7 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT public void testRateLimitingMultipleConsumers() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingMultipleConsumers"; final int messageRate = 5; @@ -604,7 +605,7 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { conf.setDispatchThrottlingOnBatchMessageEnabled(true); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingMultipleConsumers"; final int messageRate = 5; @@ -672,7 +673,7 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { public void testClusterRateLimitingConfiguration(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 5; @@ -735,7 +736,7 @@ public void testClusterRateLimitingConfiguration(SubscriptionType subscription) public void testMessageByteRateThrottlingCombined(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingAll"; final int messageRate = 5; // 5 msgs per second @@ -805,7 +806,7 @@ public void testMessageByteRateThrottlingCombined(SubscriptionType subscription) public void testGlobalNamespaceThrottling() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 5; @@ -871,7 +872,7 @@ public void testGlobalNamespaceThrottling() throws Exception { public void testNonBacklogConsumerWithThrottlingEnabled(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 10; @@ -950,7 +951,7 @@ public void testNonBacklogConsumerWithThrottlingEnabled(SubscriptionType subscri public void testClusterPolicyOverrideConfiguration() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName1 = "persistent://" + namespace + "/throttlingOverride1"; final String topicName2 = "persistent://" + namespace + "/throttlingOverride2"; final int clusterMessageRate = 100; @@ -1020,7 +1021,7 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { public void testClosingRateLimiter(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/closingRateLimiter" + subscription.name(); final String subName = "mySubscription" + subscription.name(); @@ -1068,7 +1069,7 @@ public void testClosingRateLimiter(SubscriptionType subscription) throws Excepti @SuppressWarnings("deprecation") @Test public void testDispatchRateCompatibility2() throws Exception { - final String namespace = "my-property/dispatch-rate-compatibility"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/dispatch-rate-compatibility"); final String topicName = "persistent://" + namespace + "/t1"; final String cluster = "test"; admin.namespaces().createNamespace(namespace, Sets.newHashSet(cluster)); @@ -1135,7 +1136,7 @@ protected void deactiveCursors(ManagedLedgerImpl ledger) throws Exception { public void testRelativeMessageRateLimitingThrottling(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/relative_throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/relative_throttling_ns"); final String topicName = "persistent://" + namespace + "/relative-throttle" + subscription; final int messageRate = 1; From 9277208f402056450d7026d5b1461ce2eaa6025c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 23:46:57 +0200 Subject: [PATCH 53/63] Extract common base class to avoid test duplication --- ...AbstractMessageDispatchThrottlingTest.java | 116 ++++++++++++++++++ .../api/MessageDispatchThrottlingTest.java | 95 +------------- ...criptionMessageDispatchThrottlingTest.java | 2 +- 3 files changed, 118 insertions(+), 95 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java new file mode 100644 index 0000000000000..1a9b1e87812e0 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.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.client.api; + +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ScheduledExecutorService; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; +import org.apache.pulsar.broker.service.BrokerService; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; + +public abstract class AbstractMessageDispatchThrottlingTest extends ProducerConsumerBase { + public static T[] merge(T[] first, T[] last) { + int totalLength = first.length + last.length; + T[] result = Arrays.copyOf(first, totalLength); + int offset = first.length; + System.arraycopy(last, 0, result, offset, first.length); + return result; + } + + @BeforeClass + @Override + protected void setup() throws Exception { + AsyncTokenBucket.switchToConsistentTokensView(); + this.conf.setClusterName("test"); + internalSetup(); + producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + internalCleanup(); + AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); + } + + @AfterMethod(alwaysRun = true) + protected void reset() throws Exception { + pulsar.getConfiguration().setForceDeleteTenantAllowed(true); + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(true); + + for (String tenant : admin.tenants().getTenants()) { + for (String namespace : admin.namespaces().getNamespaces(tenant)) { + admin.namespaces().deleteNamespace(namespace, true); + } + admin.tenants().deleteTenant(tenant, true); + } + + for (String cluster : admin.clusters().getClusters()) { + admin.clusters().deleteCluster(cluster); + } + + pulsar.getConfiguration().setForceDeleteTenantAllowed(false); + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); + + producerBaseSetup(); + } + + @DataProvider(name = "subscriptions") + public Object[][] subscriptionsProvider() { + return new Object[][]{new Object[]{SubscriptionType.Shared}, {SubscriptionType.Exclusive}}; + } + + @DataProvider(name = "dispatchRateType") + public Object[][] dispatchRateProvider() { + return new Object[][]{{DispatchRateType.messageRate}, {DispatchRateType.byteRate}}; + } + + @DataProvider(name = "subscriptionAndDispatchRateType") + public Object[][] subDisTypeProvider() { + List mergeList = new LinkedList<>(); + for (Object[] sub : subscriptionsProvider()) { + for (Object[] dispatch : dispatchRateProvider()) { + mergeList.add(AbstractMessageDispatchThrottlingTest.merge(sub, dispatch)); + } + } + return mergeList.toArray(new Object[0][0]); + } + + protected void deactiveCursors(ManagedLedgerImpl ledger) throws Exception { + Field statsUpdaterField = BrokerService.class.getDeclaredField("statsUpdater"); + statsUpdaterField.setAccessible(true); + ScheduledExecutorService statsUpdater = (ScheduledExecutorService) statsUpdaterField + .get(pulsar.getBrokerService()); + statsUpdater.shutdownNow(); + ledger.getCursors().forEach(cursor -> { + ledger.deactivateCursor(cursor); + }); + } + + enum DispatchRateType { + messageRate, byteRate; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index 6e0b38bbb24d6..b4f6eee26c7f5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -28,15 +28,11 @@ import static org.testng.Assert.fail; import com.google.common.collect.Sets; import java.lang.reflect.Field; -import java.util.Arrays; import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; @@ -45,8 +41,6 @@ import org.apache.bookkeeper.mledger.impl.cache.PendingReadsManager; import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; import org.apache.pulsar.broker.BrokerTestUtil; -import org.apache.pulsar.broker.qos.AsyncTokenBucket; -import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.common.policies.data.ClusterData; @@ -60,88 +54,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Test(groups = "broker-api") -public class MessageDispatchThrottlingTest extends ProducerConsumerBase { +public class MessageDispatchThrottlingTest extends AbstractMessageDispatchThrottlingTest { private static final Logger log = LoggerFactory.getLogger(MessageDispatchThrottlingTest.class); - @BeforeClass - @Override - protected void setup() throws Exception { - AsyncTokenBucket.switchToConsistentTokensView(); - this.conf.setClusterName("test"); - super.internalSetup(); - super.producerBaseSetup(); - } - - @AfterClass(alwaysRun = true) - @Override - protected void cleanup() throws Exception { - super.internalCleanup(); - AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); - } - - @AfterMethod(alwaysRun = true) - protected void reset() throws Exception { - pulsar.getConfiguration().setForceDeleteTenantAllowed(true); - pulsar.getConfiguration().setForceDeleteNamespaceAllowed(true); - - for (String tenant : admin.tenants().getTenants()) { - for (String namespace : admin.namespaces().getNamespaces(tenant)) { - admin.namespaces().deleteNamespace(namespace, true); - } - admin.tenants().deleteTenant(tenant, true); - } - - for (String cluster : admin.clusters().getClusters()) { - admin.clusters().deleteCluster(cluster); - } - - pulsar.getConfiguration().setForceDeleteTenantAllowed(false); - pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); - - super.producerBaseSetup(); - } - - - @DataProvider(name = "subscriptions") - public Object[][] subscriptionsProvider() { - return new Object[][] { new Object[] { SubscriptionType.Shared }, { SubscriptionType.Exclusive } }; - } - - @DataProvider(name = "dispatchRateType") - public Object[][] dispatchRateProvider() { - return new Object[][] { { DispatchRateType.messageRate }, { DispatchRateType.byteRate } }; - } - - @DataProvider(name = "subscriptionAndDispatchRateType") - public Object[][] subDisTypeProvider() { - List mergeList = new LinkedList<>(); - for (Object[] sub : subscriptionsProvider()) { - for (Object[] dispatch : dispatchRateProvider()) { - mergeList.add(merge(sub, dispatch)); - } - } - return mergeList.toArray(new Object[0][0]); - } - - public static T[] merge(T[] first, T[] last) { - int totalLength = first.length + last.length; - T[] result = Arrays.copyOf(first, totalLength); - int offset = first.length; - System.arraycopy(last, 0, result, offset, first.length); - return result; - } - - enum DispatchRateType { - messageRate, byteRate; - } - /** * verifies: message-rate change gets reflected immediately into topic at runtime * @@ -1115,17 +1033,6 @@ public void testDispatchRateCompatibility2() throws Exception { topic.close().get(); } - protected void deactiveCursors(ManagedLedgerImpl ledger) throws Exception { - Field statsUpdaterField = BrokerService.class.getDeclaredField("statsUpdater"); - statsUpdaterField.setAccessible(true); - ScheduledExecutorService statsUpdater = (ScheduledExecutorService) statsUpdaterField - .get(pulsar.getBrokerService()); - statsUpdater.shutdownNow(); - ledger.getCursors().forEach(cursor -> { - ledger.deactivateCursor(cursor); - }); - } - /** * It verifies that relative throttling at least dispatch messages as publish-rate. * diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index 88a680fe33007..db40ec644e9ca 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -41,7 +41,7 @@ import org.testng.annotations.Test; @Test(groups = "broker-api") -public class SubscriptionMessageDispatchThrottlingTest extends MessageDispatchThrottlingTest { +public class SubscriptionMessageDispatchThrottlingTest extends AbstractMessageDispatchThrottlingTest { private static final Logger log = LoggerFactory.getLogger(SubscriptionMessageDispatchThrottlingTest.class); /** From 96781e965614b847b29bd91a0ed46b1b40013db9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Feb 2025 23:51:50 +0200 Subject: [PATCH 54/63] Reduce flakiness --- .../apache/pulsar/client/api/MessageDispatchThrottlingTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index b4f6eee26c7f5..95eeb38ea8656 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -462,7 +462,7 @@ public void testRateLimitingMultipleConsumers() throws Exception { admin.namespaces().setDispatchRate(namespace, dispatchRate); // create producer and topic @Cleanup - Producer producer = pulsarClient.newProducer().topic(topicName).create(); + Producer producer = pulsarClient.newProducer().enableBatching(false).topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); Awaitility.await() From 234cfc9b74840d438f3bed96012b872b4905170b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 00:10:17 +0200 Subject: [PATCH 55/63] Refactor common config --- .../persistent/DispatchRateLimiter.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java index 5df6204f6b0bb..f43b134eb122a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.qos.AsyncTokenBucket; -import org.apache.pulsar.broker.qos.MonotonicSnapshotClock; +import org.apache.pulsar.broker.qos.AsyncTokenBucketBuilder; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -222,21 +222,18 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { long msgRate = dispatchRate.getDispatchThrottlingRateInMsg(); long byteRate = dispatchRate.getDispatchThrottlingRateInByte(); long ratePeriodNanos = TimeUnit.SECONDS.toNanos(Math.max(dispatchRate.getRatePeriodInSecond(), 1)); - MonotonicSnapshotClock clock = brokerService.getPulsar().getMonotonicSnapshotClock(); // update msg-rateLimiter if (msgRate > 0) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnMessage = - AsyncTokenBucket.builderForDynamicRate() - .clock(clock) + configureAsyncTokenBucket(AsyncTokenBucket.builderForDynamicRate()) .rateFunction(() -> getRelativeDispatchRateInMsg(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) .build(); } else { this.dispatchRateLimiterOnMessage = - AsyncTokenBucket.builder() - .clock(clock) + configureAsyncTokenBucket(AsyncTokenBucket.builder()) .rate(msgRate).ratePeriodNanos(ratePeriodNanos) .build(); } @@ -248,15 +245,13 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { if (byteRate > 0) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnByte = - AsyncTokenBucket.builderForDynamicRate() - .clock(clock) + configureAsyncTokenBucket(AsyncTokenBucket.builderForDynamicRate()) .rateFunction(() -> getRelativeDispatchRateInByte(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) .build(); } else { this.dispatchRateLimiterOnByte = - AsyncTokenBucket.builder() - .clock(clock) + configureAsyncTokenBucket(AsyncTokenBucket.builder()) .rate(byteRate).ratePeriodNanos(ratePeriodNanos) .build(); } @@ -265,6 +260,11 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { } } + private > T configureAsyncTokenBucket(T builder) { + builder.clock(brokerService.getPulsar().getMonotonicSnapshotClock()); + return builder; + } + private long getRelativeDispatchRateInMsg(DispatchRate dispatchRate) { return (topic != null && dispatchRate != null) ? (long) topic.getLastUpdatedAvgPublishRateInMsg() + dispatchRate.getDispatchThrottlingRateInMsg() From 8ef12caa003ae9084a862e8840da216d5612a2ca Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 00:18:00 +0200 Subject: [PATCH 56/63] Fix flakiness --- .../apache/pulsar/client/api/MessageDispatchThrottlingTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index 95eeb38ea8656..5d6f0c519abc6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -451,6 +451,7 @@ public void testRateLimitingMultipleConsumers() throws Exception { final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingMultipleConsumers"; + conf.setDispatchThrottlingOnNonBacklogConsumerEnabled(true); final int messageRate = 5; DispatchRate dispatchRate = DispatchRate.builder() @@ -515,6 +516,7 @@ public void testRateLimitingMultipleConsumers() throws Exception { assertThat(totalReceived.get()).isCloseTo(messageRate, Offset.offset(Math.max(messageRate / 10, 2))); log.info("-- Exiting {} test --", methodName); + conf.setDispatchThrottlingOnNonBacklogConsumerEnabled(false); } @Test From 68ba451aa12255cbfa11a07b31bfe84365f3dff4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 00:29:45 +0200 Subject: [PATCH 57/63] Move MessagePublishThrottlingTest to broker-api test group --- .../apache/pulsar/client/impl/MessagePublishThrottlingTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java index 1c0ae5547d53b..a848d68f37f63 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java @@ -41,7 +41,7 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -@Test +@Test(groups = "broker-api") public class MessagePublishThrottlingTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(MessagePublishThrottlingTest.class); From 2d99778c00e2fe3eb62c5a49c4f249f6ffc9b292 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 00:59:16 +0200 Subject: [PATCH 58/63] Fix issue in lookupUrl change in test class --- .../broker/auth/MockedPulsarServiceBaseTest.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index e46b76205ecf0..69572d693cd3e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -184,7 +184,9 @@ private URI resolveLookupUrl() { if (isTcpLookup) { return URI.create(pulsar.getBrokerServiceUrl()); } else { - return URI.create(brokerUrl.toString()); + return URI.create(brokerUrl != null + ? brokerUrl.toString() + : brokerUrlTls.toString()); } } @@ -232,11 +234,10 @@ protected PulsarClient replacePulsarClient(ClientBuilder clientBuilder) throws P protected final void internalSetupForStatsTest() throws Exception { init(); - String lookupUrl = brokerUrl.toString(); - if (isTcpLookup) { - lookupUrl = new URI(pulsar.getBrokerServiceUrl()).toString(); + if (pulsarClient != null) { + pulsarClient.shutdown(); } - pulsarClient = newPulsarClient(lookupUrl, 1); + pulsarClient = newPulsarClient(resolveLookupUrl().toString(), 1); } protected void doInitConf() throws Exception { From c67ceb022c98e65be474cbfbf933c6a3203c79f4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 01:05:15 +0200 Subject: [PATCH 59/63] Revisit startBroker method in test base class --- .../pulsar/broker/auth/MockedPulsarServiceBaseTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index 69572d693cd3e..376f997b27149 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -390,9 +390,11 @@ protected void startBroker() throws Exception { brokerUrlTls = pulsar.getWebServiceAddressTls() != null ? new URL(pulsar.getWebServiceAddressTls()) : null; URI newLookupUrl = resolveLookupUrl(); - if (pulsarClient != null && (lookupUrl == null || !newLookupUrl.equals(lookupUrl))) { - pulsarClient.shutdown(); + if (lookupUrl == null || !newLookupUrl.equals(lookupUrl)) { lookupUrl = newLookupUrl; + } + if (pulsarClient != null) { + pulsarClient.shutdown(); pulsarClient = newPulsarClient(lookupUrl.toString(), 0); } From 6de794b5069a96df67825565a14567910d7790d0 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 01:11:13 +0200 Subject: [PATCH 60/63] Revisit logic one more time in test class --- .../pulsar/broker/auth/MockedPulsarServiceBaseTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index 376f997b27149..d23bcf27adbde 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -392,10 +392,10 @@ protected void startBroker() throws Exception { URI newLookupUrl = resolveLookupUrl(); if (lookupUrl == null || !newLookupUrl.equals(lookupUrl)) { lookupUrl = newLookupUrl; - } - if (pulsarClient != null) { - pulsarClient.shutdown(); - pulsarClient = newPulsarClient(lookupUrl.toString(), 0); + if (pulsarClient != null) { + pulsarClient.shutdown(); + pulsarClient = newPulsarClient(lookupUrl.toString(), 0); + } } closeAdmin(); From f9aab2e5f99d1d1e46030b355afa913204695f02 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 11:55:38 +0200 Subject: [PATCH 61/63] Refactor consistency settings in AsyncTokenBucket and add Javadocs --- .../pulsar/broker/qos/AsyncTokenBucket.java | 113 ++++++++++-------- .../broker/qos/AsyncTokenBucketBuilder.java | 34 +++++- .../qos/DynamicRateAsyncTokenBucket.java | 9 +- .../DynamicRateAsyncTokenBucketBuilder.java | 6 +- .../broker/qos/FinalRateAsyncTokenBucket.java | 8 +- .../qos/FinalRateAsyncTokenBucketBuilder.java | 2 +- .../persistent/SubscribeRateLimiter.java | 3 +- .../broker/qos/AsyncTokenBucketTest.java | 33 ++--- 8 files changed, 123 insertions(+), 85 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index 4f9e35ca3d8d9..8c43fa0a816fa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -42,6 +42,10 @@ * connection or client from the throttling queue to unthrottle. Before unthrottling, the application should check * for available tokens. If tokens are still not available, the application should continue with throttling and * repeat the throttling loop. + *

By default, the AsyncTokenBucket is eventually consistent. This means that the token balance is updated + * with added tokens and consumed tokens at most once during each "increment", when time advances more than the + * configured resolution. There are settings for configuring consistency, please see {@link AsyncTokenBucketBuilder} + * for details. *

This class does not produce side effects outside its own scope. It functions similarly to a stateful function, * akin to a counter function. In essence, it is a sophisticated counter. It can serve as a foundational component for * constructing higher-level asynchronous rate limiter implementations, which require side effects for throttling. @@ -119,14 +123,28 @@ public static void resetToDefaultEventualConsistentTokensView() { */ private final LongAdder pendingConsumedTokens = new LongAdder(); - private final boolean consistentTokensView; + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the consumed tokens are subtracted from + * the total amount of tokens at most once during each "increment", when time advances more than the configured + * resolution. This setting determines if the consumed tokens are subtracted from tokens balance consistently. + * For high performance, it is recommended to keep this setting as false. + */ + private final boolean consistentConsumedTokens; + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the added tokens are calculated based + * on elapsed time at most once during each "increment", when time advances more than the configured + * resolution. This setting determines if the added tokens are calculated and added to tokens balance consistently. + * For high performance, it is recommended to keep this setting as false. + */ + private final boolean consistentAddedTokens; protected AsyncTokenBucket(MonotonicSnapshotClock clockSource, long resolutionNanos, - boolean consistentTokensView) { + boolean consistentConsumedTokens, boolean consistentAddedTokens) { this.clockSource = clockSource; this.resolutionNanos = resolutionNanos; this.lastNanos = Long.MIN_VALUE; - this.consistentTokensView = consistentTokensView; + this.consistentConsumedTokens = consistentConsumedTokens; + this.consistentAddedTokens = consistentAddedTokens; } public static FinalRateAsyncTokenBucketBuilder builder() { @@ -144,40 +162,46 @@ public static DynamicRateAsyncTokenBucketBuilder builderForDynamicRate() { /** * Consumes tokens and possibly updates the tokens balance. New tokens are calculated and added to the current * tokens balance each time the update takes place. The update takes place once in every interval of the configured - * resolutionNanos or when the forceUpdateTokens parameter is true. + * resolutionNanos or when the forceConsistentTokens parameter is true. * When the tokens balance isn't updated, the consumed tokens are added to the pendingConsumedTokens LongAdder * counter which gets flushed the next time the tokens are updated. This makes the tokens balance * eventually consistent. The reason for this design choice is to optimize performance by preventing CAS loop * contention which could cause excessive CPU consumption. * * @param consumeTokens number of tokens to consume, can be 0 to update the tokens balance - * @param forceUpdateTokens if true, the tokens are updated even if the configured resolution hasn't passed + * @param forceConsistentTokens if true, the token balance is updated consistently * @return the current number of tokens in the bucket or Long.MIN_VALUE when the number of tokens is unknown due * to eventual consistency */ - private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolean forceUpdateTokens) { + private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolean forceConsistentTokens) { if (consumeTokens < 0) { throw new IllegalArgumentException("consumeTokens must be >= 0"); } - long currentNanos = clockSource.getTickNanos(forceUpdateTokens); + boolean requestConsistentTickNanosSnapshot = + consistentAddedTokens || consistentConsumedTokens || forceConsistentTokens || resolutionNanos == 0; + long currentNanos = clockSource.getTickNanos(requestConsistentTickNanosSnapshot); long newTokens = 0; // check if the tokens should be updated immediately - if (shouldUpdateTokensImmediately(currentNanos, forceUpdateTokens)) { + if (shouldAddTokensImmediately(currentNanos, forceConsistentTokens)) { // calculate the number of new tokens since the last update - newTokens = calculateNewTokensSinceLastUpdate(currentNanos); + newTokens = calculateNewTokensSinceLastUpdate(currentNanos, forceConsistentTokens); } // update tokens if there are new tokens or if resolutionNanos is set to 0 which is currently used for testing - if (newTokens > 0 || resolutionNanos == 0) { + if (newTokens > 0 || resolutionNanos == 0 || consistentConsumedTokens || forceConsistentTokens) { // flush the pendingConsumedTokens by calling "sumThenReset" long currentPendingConsumedTokens = pendingConsumedTokens.sumThenReset(); // calculate the token delta by subtracting the consumed tokens from the new tokens long tokenDelta = newTokens - currentPendingConsumedTokens; - // update the tokens and return the current token value - return TOKENS_UPDATER.updateAndGet(this, - // limit the tokens to the capacity of the bucket - currentTokens -> Math.min(currentTokens + tokenDelta, getCapacity()) - // subtract the consumed tokens from the capped tokens - - consumeTokens); + if (tokenDelta != 0 || consumeTokens != 0) { + // update the tokens and return the current token value + return TOKENS_UPDATER.updateAndGet(this, + // limit the tokens to the capacity of the bucket + currentTokens -> Math.min(currentTokens + tokenDelta, getCapacity()) + // subtract the consumed tokens from the capped tokens + - consumeTokens); + } else { + return tokens; + } } else { // eventual consistent fast path, tokens are not updated immediately @@ -186,13 +210,8 @@ private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolea pendingConsumedTokens.add(consumeTokens); } - if (forceUpdateTokens) { - // return the current tokens balance without updating the tokens and resetting the pendingConsumedTokens - return tokens - pendingConsumedTokens.sum(); - } else { - // return Long.MIN_VALUE if the current value of tokens is unknown due to the eventual consistency - return Long.MIN_VALUE; - } + // return Long.MIN_VALUE if the current value of tokens is unknown due to the eventual consistency + return Long.MIN_VALUE; } } @@ -201,19 +220,19 @@ private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolea * * The tokens will be updated once every resolutionNanos nanoseconds. * This method checks if the configured resolutionNanos has passed since the last update. - * If the forceUpdateTokens is true, the tokens will be updated immediately. + * If the forceConsistentTokens is true, the tokens will be updated immediately. * - * @param currentNanos the current monotonic clock time in nanoseconds - * @param forceUpdateTokens if true, the tokens will be updated immediately + * @param currentNanos the current monotonic clock time in nanoseconds + * @param forceConsistentTokens if true, the tokens are added even if the configured resolution hasn't fully passed * @return true if the tokens should be updated immediately, false otherwise */ - private boolean shouldUpdateTokensImmediately(long currentNanos, boolean forceUpdateTokens) { + private boolean shouldAddTokensImmediately(long currentNanos, boolean forceConsistentTokens) { long currentIncrement = resolutionNanos != 0 ? currentNanos / resolutionNanos : 0; long currentLastIncrement = lastIncrement; return currentIncrement == 0 || (currentIncrement > currentLastIncrement && LAST_INCREMENT_UPDATER.compareAndSet(this, currentLastIncrement, currentIncrement)) - || forceUpdateTokens; + || consistentAddedTokens || forceConsistentTokens; } /** @@ -223,11 +242,13 @@ private boolean shouldUpdateTokensImmediately(long currentNanos, boolean forceUp * @param currentNanos the current monotonic clock time in nanoseconds * @return the number of new tokens to add since the last update */ - private long calculateNewTokensSinceLastUpdate(long currentNanos) { + private long calculateNewTokensSinceLastUpdate(long currentNanos, boolean forceConsistentTokens) { long previousLastNanos = lastNanos; long newLastNanos; // update lastNanos only if at least resolutionNanos/2 nanoseconds has passed since the last update - if (currentNanos >= previousLastNanos + resolutionNanos / 2) { + // unless consistency is needed + long minimumIncrementNanos = forceConsistentTokens || consistentAddedTokens ? 0L : resolutionNanos / 2; + if (currentNanos > previousLastNanos + minimumIncrementNanos) { newLastNanos = currentNanos; } else { newLastNanos = previousLastNanos; @@ -291,15 +312,14 @@ public boolean consumeTokensAndCheckIfContainsTokens(long consumeTokens) { } /** - * Returns the current token balance. When forceUpdateTokens is true, the tokens balance is updated before - * returning. If forceUpdateTokens is false, the tokens balance could be updated if the last updated happened + * Returns the current token balance. When forceConsistentTokens is true, the tokens balance is updated before + * returning. If forceConsistentTokens is false, the tokens balance could be updated if the last updated happened * more than resolutionNanos nanoseconds ago. * - * @param forceUpdateTokens if true, the tokens balance is updated before returning * @return the current token balance */ - protected long tokens(boolean forceUpdateTokens) { - long currentTokens = consumeTokensAndMaybeUpdateTokensBalance(0, forceUpdateTokens); + private long tokens() { + long currentTokens = consumeTokensAndMaybeUpdateTokensBalance(0, false); if (currentTokens != Long.MIN_VALUE) { // when currentTokens isn't Long.MIN_VALUE, the current tokens balance is known return currentTokens; @@ -319,7 +339,7 @@ public long calculateThrottlingDuration() { long currentTokens = consumeTokensAndMaybeUpdateTokensBalance(0, true); if (currentTokens == Long.MIN_VALUE) { throw new IllegalArgumentException( - "Unexpected result from updateAndConsumeTokens with forceUpdateTokens set to true"); + "Unexpected result from updateAndConsumeTokens with forceConsistentTokens set to true"); } if (currentTokens > 0) { return 0L; @@ -334,10 +354,10 @@ public long calculateThrottlingDuration() { /** * Returns the current number of tokens in the bucket. * The token balance is updated if the configured resolutionNanos has passed since the last update unless - * consistentTokensView is true. + * consistentConsumedTokens is true. */ public final long getTokens() { - return tokens(consistentTokensView); + return tokens(); } public abstract long getRate(); @@ -346,25 +366,12 @@ public final long getTokens() { * Checks if the bucket contains tokens. * The token balance is updated before the comparison if the configured resolutionNanos has passed since the last * update. It's possible that the returned result is not definite since the token balance is eventually consistent - * if consistentTokensView is false. + * if consistentConsumedTokens is false. * * @return true if the bucket contains tokens, false otherwise */ public boolean containsTokens() { - return containsTokens(consistentTokensView); - } - - /** - * Checks if the bucket contains tokens. - * The token balance is updated before the comparison if the configured resolutionNanos has passed since the last - * update. The token balance is also updated when forceUpdateTokens is true. - * It's possible that the returned result is not definite since the token balance is eventually consistent. - * - * @param forceUpdateTokens if true, the token balance is updated before the comparison - * @return true if the bucket contains tokens, false otherwise - */ - protected boolean containsTokens(boolean forceUpdateTokens) { - return tokens(forceUpdateTokens) > 0; + return tokens() > 0; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java index cad90561a13a5..1c05f1a213e3a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java @@ -23,7 +23,8 @@ public abstract class AsyncTokenBucketBuilder> { protected MonotonicSnapshotClock clock = AsyncTokenBucket.DEFAULT_SNAPSHOT_CLOCK; protected long resolutionNanos = AsyncTokenBucket.defaultResolutionNanos; - protected boolean consistentTokensView; + protected boolean consistentConsumedTokens; + protected boolean consistentAddedTokens; protected AsyncTokenBucketBuilder() { } @@ -32,18 +33,45 @@ protected SELF self() { return (SELF) this; } + /** + * Set the clock source for the token bucket. It's recommended to use the {@link DefaultMonotonicSnapshotClock} + * for most use cases. + */ public SELF clock(MonotonicSnapshotClock clock) { this.clock = clock; return self(); } + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the token balance is updated, when time + * advances more than the configured resolution. This setting determines the duration of the increment. + * Setting this value to 0 will make the token balance fully consistent. There's a performance trade-off + * when setting this value to 0. + */ public SELF resolutionNanos(long resolutionNanos) { this.resolutionNanos = resolutionNanos; return self(); } - public SELF consistentTokensView(boolean consistentTokensView) { - this.consistentTokensView = consistentTokensView; + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the consumed tokens are subtracted from + * the total amount of tokens at most once during each "increment", when time advances more than the configured + * resolution. This setting determines if the consumed tokens are subtracted from tokens balance consistently. + * For high performance, it is recommended to keep this setting as false. + */ + public SELF consistentConsumedTokens(boolean consistentConsumedTokens) { + this.consistentConsumedTokens = consistentConsumedTokens; + return self(); + } + + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the added tokens are calculated based + * on elapsed time at most once during each "increment", when time advances more than the configured + * resolution. This setting determines if the added tokens are calculated and added to tokens balance consistently. + * For high performance, it is recommended to keep this setting as false. + */ + public SELF consistentAddedTokens(boolean consistentAddedTokens) { + this.consistentAddedTokens = consistentAddedTokens; return self(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java index 05ac138d0289a..f2eae8aed8d9c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java @@ -34,15 +34,16 @@ public class DynamicRateAsyncTokenBucket extends AsyncTokenBucket { protected DynamicRateAsyncTokenBucket(double capacityFactor, LongSupplier rateFunction, MonotonicSnapshotClock clockSource, LongSupplier ratePeriodNanosFunction, - long resolutionNanos, boolean consistentTokensView, - double initialTokensFactor, double targetFillFactorAfterThrottling) { - super(clockSource, resolutionNanos, consistentTokensView); + long resolutionNanos, boolean consistentConsumedTokens, + boolean consistentAddedTokens, double initialTokensFactor, + double targetFillFactorAfterThrottling) { + super(clockSource, resolutionNanos, consistentConsumedTokens, consistentAddedTokens); this.capacityFactor = capacityFactor; this.rateFunction = rateFunction; this.ratePeriodNanosFunction = ratePeriodNanosFunction; this.targetFillFactorAfterThrottling = targetFillFactorAfterThrottling; this.tokens = (long) (rateFunction.getAsLong() * initialTokensFactor); - tokens(false); + getTokens(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java index ce38ff2a4078c..8aebecddf90c7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java @@ -64,9 +64,7 @@ public DynamicRateAsyncTokenBucketBuilder targetFillFactorAfterThrottling( @Override public AsyncTokenBucket build() { return new DynamicRateAsyncTokenBucket(this.capacityFactor, this.rateFunction, - this.clock, - this.ratePeriodNanosFunction, this.resolutionNanos, this.consistentTokensView, - this.initialFillFactor, - targetFillFactorAfterThrottling); + this.clock, this.ratePeriodNanosFunction, this.resolutionNanos, this.consistentConsumedTokens, + this.consistentAddedTokens, this.initialFillFactor, targetFillFactorAfterThrottling); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java index 3f5edcbf4f06d..d83290b723f07 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java @@ -30,16 +30,16 @@ class FinalRateAsyncTokenBucket extends AsyncTokenBucket { private final long targetAmountOfTokensAfterThrottling; protected FinalRateAsyncTokenBucket(long capacity, long rate, MonotonicSnapshotClock clockSource, - long ratePeriodNanos, long resolutionNanos, boolean consistentTokensView, - long initialTokens) { - super(clockSource, resolutionNanos, consistentTokensView); + long ratePeriodNanos, long resolutionNanos, boolean consistentConsumedTokens, + boolean consistentAddedTokens, long initialTokens) { + super(clockSource, resolutionNanos, consistentConsumedTokens, consistentAddedTokens); this.capacity = capacity; this.rate = rate; this.ratePeriodNanos = ratePeriodNanos != -1 ? ratePeriodNanos : ONE_SECOND_NANOS; // The target amount of tokens is the amount of tokens made available in the resolution duration this.targetAmountOfTokensAfterThrottling = Math.max(this.resolutionNanos * rate / ratePeriodNanos, 1); this.tokens = initialTokens; - tokens(false); + getTokens(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java index 848eaaaca0b49..a292000eaa825 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java @@ -55,7 +55,7 @@ public FinalRateAsyncTokenBucketBuilder initialTokens(long initialTokens) { public AsyncTokenBucket build() { return new FinalRateAsyncTokenBucket(this.capacity != null ? this.capacity : this.rate, this.rate, this.clock, - this.ratePeriodNanos, this.resolutionNanos, this.consistentTokensView, + this.ratePeriodNanos, this.resolutionNanos, this.consistentConsumedTokens, this.consistentAddedTokens, this.initialTokens != null ? this.initialTokens : this.rate ); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java index c465a36359bd2..0f98ab94142c8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java @@ -118,7 +118,8 @@ private synchronized void updateSubscribeRate(ConsumerIdentifier consumerIdentif if (ratePerConsumer > 0) { AsyncTokenBucket tokenBucket = AsyncTokenBucket.builder() - .consistentTokensView(true) + .consistentAddedTokens(true) + .consistentConsumedTokens(true) .clock(brokerService.getPulsar().getMonotonicSnapshotClock()) .rate(ratePerConsumer).ratePeriodNanos(ratePeriodNanos).build(); this.subscribeRateLimiter.put(consumerIdentifier, tokenBucket); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index 142320ac52804..82793f2748d78 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -51,7 +51,8 @@ private void incrementMillis(long millis) { @Test void shouldAddTokensWithConfiguredRate() { asyncTokenBucket = - AsyncTokenBucket.builder().capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + AsyncTokenBucket.builder().consistentConsumedTokens(true) + .capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); incrementSeconds(5); assertEquals(asyncTokenBucket.getTokens(), 50); incrementSeconds(1); @@ -65,7 +66,7 @@ void shouldAddTokensWithConfiguredRate() { // Consume all and verify none available and then wait 1 period and check replenished asyncTokenBucket.consumeTokens(100); - assertEquals(asyncTokenBucket.tokens(true), 0); + assertEquals(asyncTokenBucket.getTokens(), 0); incrementSeconds(1); assertEquals(asyncTokenBucket.getTokens(), 10); } @@ -121,16 +122,16 @@ void shouldSupportFractionsAndRetainLeftoverWhenUpdatingTokens2() { .build(); for (int i = 0; i < 150; i++) { incrementMillis(1); - assertEquals(asyncTokenBucket.tokens((i + 1) % 31 == 0), 0); + assertEquals(asyncTokenBucket.getTokens(), 0); } incrementMillis(150); - assertEquals(asyncTokenBucket.tokens(true), 0); + assertEquals(asyncTokenBucket.getTokens(), 0); incrementMillis(699); - assertEquals(asyncTokenBucket.tokens(true), 0); + assertEquals(asyncTokenBucket.getTokens(), 0); incrementMillis(1); - assertEquals(asyncTokenBucket.tokens(true), 1); + assertEquals(asyncTokenBucket.getTokens(), 1); incrementMillis(1000); - assertEquals(asyncTokenBucket.tokens(true), 2); + assertEquals(asyncTokenBucket.getTokens(), 2); } @Test @@ -141,7 +142,7 @@ void shouldHandleNegativeBalanceWithEventuallyConsistentTokenUpdates() { .resolutionNanos(TimeUnit.SECONDS.toNanos(51)) .capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); // assert that the token balance is 0 initially - assertThat(asyncTokenBucket.tokens(true)).isEqualTo(0); + assertThat(asyncTokenBucket.getTokens()).isEqualTo(0); // consume tokens without exceeding the rate for (int i = 0; i < 10000; i++) { @@ -153,7 +154,7 @@ void shouldHandleNegativeBalanceWithEventuallyConsistentTokenUpdates() { incrementSeconds(9); // there should be 90 tokens available - assertThat(asyncTokenBucket.tokens(true)).isEqualTo(90); + assertThat(asyncTokenBucket.getTokens()).isEqualTo(90); } @Test @@ -164,7 +165,7 @@ void shouldNotExceedTokenBucketSizeWithNegativeTokens() { .resolutionNanos(TimeUnit.SECONDS.toNanos(51)) .capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); // assert that the token balance is 0 initially - assertThat(asyncTokenBucket.tokens(true)).isEqualTo(0); + assertThat(asyncTokenBucket.getTokens()).isEqualTo(0); // consume tokens without exceeding the rate for (int i = 0; i < 100; i++) { @@ -180,7 +181,7 @@ void shouldNotExceedTokenBucketSizeWithNegativeTokens() { incrementSeconds(9); // there should be 90 tokens available - assertThat(asyncTokenBucket.tokens(true)).isEqualTo(90); + assertThat(asyncTokenBucket.getTokens()).isEqualTo(90); } @Test @@ -197,7 +198,7 @@ void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates asyncTokenBucket = AsyncTokenBucket.builder().resolutionNanos(TimeUnit.SECONDS.toNanos(51)) .capacity(100).rate(10).initialTokens(100).clock(clockSource).build(); - assertThat(asyncTokenBucket.tokens(true)).isEqualTo(100); + assertThat(asyncTokenBucket.getTokens()).isEqualTo(100); // consume tokens without exceeding the rate for (int i = 0; i < 10000; i++) { @@ -209,7 +210,7 @@ void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates incrementSeconds(9); // there should be 90 tokens available - assertThat(asyncTokenBucket.tokens(true)).isEqualTo(90); + assertThat(asyncTokenBucket.getTokens()).isEqualTo(90); } @Test @@ -221,7 +222,9 @@ void shouldHandleEventualConsistency() { () -> offset.get() + manualClockSource.get()); long initialTokens = 500L; asyncTokenBucket = - AsyncTokenBucket.builder().resolutionNanos(resolutionNanos) + AsyncTokenBucket.builder() + .consistentConsumedTokens(true) + .resolutionNanos(resolutionNanos) .capacity(100000).rate(1000).initialTokens(initialTokens).clock(monotonicSnapshotClock).build(); for (int i = 0; i < 100000; i++) { // increment the clock by 1ms, since rate is 1000 tokens/s, this should make 1 token available @@ -229,7 +232,7 @@ void shouldHandleEventualConsistency() { // consume 1 token asyncTokenBucket.consumeTokens(1); } - assertThat(asyncTokenBucket.tokens(true)) + assertThat(asyncTokenBucket.getTokens()) // since the rate is 1/ms and the test increments the clock by 1ms and consumes 1 token in each // iteration, the tokens should be equal to the initial tokens .isEqualTo(initialTokens); From e4f4689b6fc0901f0e62b2048174f85c97918db8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 12:48:49 +0200 Subject: [PATCH 62/63] Attempt to fix flaky test MessageDispatchThrottlingTest --- .../apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java | 3 +++ .../client/api/AbstractMessageDispatchThrottlingTest.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index d23bcf27adbde..42e2c00f73acf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -365,6 +365,9 @@ protected void afterPulsarStart(PulsarService pulsar) throws Exception { protected void restartBroker() throws Exception { stopBroker(); startBroker(); + if (pulsarClient == null) { + pulsarClient = newPulsarClient(lookupUrl.toString(), 0); + } } protected void stopBroker() throws Exception { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java index 1a9b1e87812e0..31c628b2bc4ca 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java @@ -40,7 +40,7 @@ public static T[] merge(T[] first, T[] last) { return result; } - @BeforeClass + @BeforeClass(alwaysRun = true) @Override protected void setup() throws Exception { AsyncTokenBucket.switchToConsistentTokensView(); From 42fb876b873dc7e77766a6f7918a88b3dc18244b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 13:32:43 +0200 Subject: [PATCH 63/63] Use consistent tokens view in flaky RGUsageMTAggrWaitForAllMsgsTest --- .../resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java index 392ec0d3ff46f..8343680f9bf7b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.BytesAndMessagesCount; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.ResourceGroupMonitoringClass; import org.apache.pulsar.broker.resourcegroup.ResourceGroupService.ResourceGroupUsageStatsType; @@ -58,9 +59,10 @@ @Slf4j @Test(groups = "flaky") public class RGUsageMTAggrWaitForAllMsgsTest extends ProducerConsumerBase { - @BeforeClass + @BeforeClass(alwaysRun = true) @Override protected void setup() throws Exception { + AsyncTokenBucket.switchToConsistentTokensView(); super.internalSetup(); this.prepareForOps(); @@ -91,6 +93,7 @@ public long computeLocalQuota(long confUsage, long myUsage, long[] allUsages) { @Override protected void cleanup() throws Exception { super.internalCleanup(); + AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); } @Test