From 281baf7c374f91673ecfa9e3e024b8889153886c Mon Sep 17 00:00:00 2001 From: Loic Rouchon Date: Thu, 5 Dec 2024 15:49:39 +0100 Subject: [PATCH 1/9] Dynamically configure SemaphoreBackPressureHandler with BackPressureLimiter (#1251) --- .../listener/AbstractContainerOptions.java | 38 +++ ...tractPipelineMessageListenerContainer.java | 11 +- .../sqs/listener/BackPressureLimiter.java | 44 +++ .../cloud/sqs/listener/ContainerOptions.java | 16 +- .../sqs/listener/ContainerOptionsBuilder.java | 18 ++ .../SemaphoreBackPressureHandler.java | 167 ++++++++-- .../sqs/integration/SqsIntegrationTests.java | 301 ++++++++++++++++++ 7 files changed, 562 insertions(+), 33 deletions(-) create mode 100644 spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureLimiter.java diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractContainerOptions.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractContainerOptions.java index 81f4eb3f2..2662b0ade 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractContainerOptions.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractContainerOptions.java @@ -47,12 +47,16 @@ public abstract class AbstractContainerOptions, private final Duration maxDelayBetweenPolls; + private final Duration standbyLimitPollingInterval; + private final Duration listenerShutdownTimeout; private final Duration acknowledgementShutdownTimeout; private final BackPressureMode backPressureMode; + private final BackPressureLimiter backPressureLimiter; + private final ListenerMode listenerMode; private final MessagingMessageConverter messageConverter; @@ -80,10 +84,12 @@ protected AbstractContainerOptions(Builder builder) { this.autoStartup = builder.autoStartup; this.pollTimeout = builder.pollTimeout; this.pollBackOffPolicy = builder.pollBackOffPolicy; + this.standbyLimitPollingInterval = builder.standbyLimitPollingInterval; this.maxDelayBetweenPolls = builder.maxDelayBetweenPolls; this.listenerShutdownTimeout = builder.listenerShutdownTimeout; this.acknowledgementShutdownTimeout = builder.acknowledgementShutdownTimeout; this.backPressureMode = builder.backPressureMode; + this.backPressureLimiter = builder.backPressureLimiter; this.listenerMode = builder.listenerMode; this.messageConverter = builder.messageConverter; this.acknowledgementMode = builder.acknowledgementMode; @@ -122,6 +128,11 @@ public BackOffPolicy getPollBackOffPolicy() { return this.pollBackOffPolicy; } + @Override + public Duration getStandbyLimitPollingInterval() { + return this.standbyLimitPollingInterval; + } + @Override public Duration getMaxDelayBetweenPolls() { return this.maxDelayBetweenPolls; @@ -154,6 +165,11 @@ public BackPressureMode getBackPressureMode() { return this.backPressureMode; } + @Override + public BackPressureLimiter getBackPressureLimiter() { + return this.backPressureLimiter; + } + @Override public ListenerMode getListenerMode() { return this.listenerMode; @@ -206,6 +222,8 @@ protected abstract static class Builder, private static final BackOffPolicy DEFAULT_POLL_BACK_OFF_POLICY = buildDefaultBackOffPolicy(); + private static final Duration DEFAULT_STANDBY_LIMIT_POLLING_INTERVAL = Duration.ofMillis(100); + private static final Duration DEFAULT_SEMAPHORE_TIMEOUT = Duration.ofSeconds(10); private static final Duration DEFAULT_LISTENER_SHUTDOWN_TIMEOUT = Duration.ofSeconds(20); @@ -214,6 +232,8 @@ protected abstract static class Builder, private static final BackPressureMode DEFAULT_THROUGHPUT_CONFIGURATION = BackPressureMode.AUTO; + private static final BackPressureLimiter DEFAULT_BACKPRESSURE_LIMITER = null; + private static final ListenerMode DEFAULT_MESSAGE_DELIVERY_STRATEGY = ListenerMode.SINGLE_MESSAGE; private static final MessagingMessageConverter DEFAULT_MESSAGE_CONVERTER = new SqsMessagingMessageConverter(); @@ -230,10 +250,14 @@ protected abstract static class Builder, private BackOffPolicy pollBackOffPolicy = DEFAULT_POLL_BACK_OFF_POLICY; + private Duration standbyLimitPollingInterval = DEFAULT_STANDBY_LIMIT_POLLING_INTERVAL; + private Duration maxDelayBetweenPolls = DEFAULT_SEMAPHORE_TIMEOUT; private BackPressureMode backPressureMode = DEFAULT_THROUGHPUT_CONFIGURATION; + private BackPressureLimiter backPressureLimiter = DEFAULT_BACKPRESSURE_LIMITER; + private Duration listenerShutdownTimeout = DEFAULT_LISTENER_SHUTDOWN_TIMEOUT; private Duration acknowledgementShutdownTimeout = DEFAULT_ACKNOWLEDGEMENT_SHUTDOWN_TIMEOUT; @@ -272,6 +296,7 @@ protected Builder(AbstractContainerOptions options) { this.listenerShutdownTimeout = options.listenerShutdownTimeout; this.acknowledgementShutdownTimeout = options.acknowledgementShutdownTimeout; this.backPressureMode = options.backPressureMode; + this.backPressureLimiter = options.backPressureLimiter; this.listenerMode = options.listenerMode; this.messageConverter = options.messageConverter; this.acknowledgementMode = options.acknowledgementMode; @@ -315,6 +340,13 @@ public B pollBackOffPolicy(BackOffPolicy pollBackOffPolicy) { return self(); } + @Override + public B standbyLimitPollingInterval(Duration standbyLimitPollingInterval) { + Assert.notNull(standbyLimitPollingInterval, "standbyLimitPollingInterval cannot be null"); + this.standbyLimitPollingInterval = standbyLimitPollingInterval; + return self(); + } + @Override public B maxDelayBetweenPolls(Duration maxDelayBetweenPolls) { Assert.notNull(maxDelayBetweenPolls, "semaphoreAcquireTimeout cannot be null"); @@ -364,6 +396,12 @@ public B backPressureMode(BackPressureMode backPressureMode) { return self(); } + @Override + public B backPressureLimiter(BackPressureLimiter backPressureLimiter) { + this.backPressureLimiter = backPressureLimiter; + return self(); + } + @Override public B acknowledgementInterval(Duration acknowledgementInterval) { Assert.notNull(acknowledgementInterval, "acknowledgementInterval cannot be null"); diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java index 6808f647a..79073c96c 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java @@ -225,10 +225,13 @@ private TaskExecutor validateCustomExecutor(TaskExecutor taskExecutor) { } protected BackPressureHandler createBackPressureHandler() { - return SemaphoreBackPressureHandler.builder().batchSize(getContainerOptions().getMaxMessagesPerPoll()) - .totalPermits(getContainerOptions().getMaxConcurrentMessages()) - .acquireTimeout(getContainerOptions().getMaxDelayBetweenPolls()) - .throughputConfiguration(getContainerOptions().getBackPressureMode()).build(); + O containerOptions = getContainerOptions(); + return SemaphoreBackPressureHandler.builder().batchSize(containerOptions.getMaxMessagesPerPoll()) + .totalPermits(containerOptions.getMaxConcurrentMessages()) + .standbyLimitPollingInterval(containerOptions.getStandbyLimitPollingInterval()) + .acquireTimeout(containerOptions.getMaxDelayBetweenPolls()) + .throughputConfiguration(containerOptions.getBackPressureMode()) + .backPressureLimiter(containerOptions.getBackPressureLimiter()).build(); } protected TaskExecutor createSourcesTaskExecutor() { diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureLimiter.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureLimiter.java new file mode 100644 index 000000000..f85ddba82 --- /dev/null +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureLimiter.java @@ -0,0 +1,44 @@ +/* + * Copyright 2013-2024 the original author or authors. + * + * Licensed 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 + * + * https://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 io.awspring.cloud.sqs.listener; + +/** + * The BackPressureLimiter enables a dynamic reduction of the queues consumption capacity depending on external factors. + */ +public interface BackPressureLimiter { + + /** + * {@return the limit to be applied to the queue consumption.} + * + * The limit can be used to reduce the queue consumption capabilities of the next polling attempts. The container + * will work toward satisfying the limit by decreasing the maximum number of concurrent messages that can ve + * processed. + * + * The following values will have the following effects: + * + *
    + *
  • zero or negative limits will stop consumption from the queue. When such a situation occurs, the queue + * processing is said to be on "standby".
  • + *
  • Values >= 1 and < {@link ContainerOptions#getMaxConcurrentMessages()} will reduce the queue consumption + * capabilities of the next polling attempts.
  • + *
  • Values >= {@link ContainerOptions#getMaxConcurrentMessages()} will not reduce the queue consumption + * capabilities
  • + *
+ * + * Note: the adjustment will require a few polling cycles to be in effect. + */ + int limit(); +} diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptions.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptions.java index ad7313cf6..e78f967a6 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptions.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptions.java @@ -59,7 +59,15 @@ public interface ContainerOptions, B extends Co boolean isAutoStartup(); /** - * Set the maximum time the polling thread should wait for a full batch of permits to be available before trying to + * {@return the amount of time to wait before checking again for the current limit when the queue processing is on + * standby} Default is 100 milliseconds. + * + * @see BackPressureLimiter#limit() + */ + Duration getStandbyLimitPollingInterval(); + + /** + * Sets the maximum time the polling thread should wait for a full batch of permits to be available before trying to * acquire a partial batch if so configured. A poll is only actually executed if at least one permit is available. * Default is 10 seconds. * @@ -127,6 +135,12 @@ default BackOffPolicy getPollBackOffPolicy() { */ BackPressureMode getBackPressureMode(); + /** + * Return the {@link BackPressureLimiter} for this container. + * @return the backpressure limiter. + */ + BackPressureLimiter getBackPressureLimiter(); + /** * Return the {@link ListenerMode} mode for this container. * @return the listener mode. diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java index 9d03b7964..de88c0464 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java @@ -56,6 +56,16 @@ public interface ContainerOptionsBuilder */ B autoStartup(boolean autoStartup); + /** + * Sets the amount of time to wait before checking again for the current limit when the queue processing is on + * standby. + * + * @param standbyLimitPollingInterval the limit polling interval when the queue processing is on standby. + * @return this instance. + * @see BackPressureLimiter#limit() + */ + B standbyLimitPollingInterval(Duration standbyLimitPollingInterval); + /** * Set the maximum time the polling thread should wait for a full batch of permits to be available before trying to * acquire a partial batch if so configured. A poll is only actually executed if at least one permit is available. @@ -145,6 +155,14 @@ default B pollBackOffPolicy(BackOffPolicy pollBackOffPolicy) { */ B backPressureMode(BackPressureMode backPressureMode); + /** + * Set the {@link BackPressureLimiter} for this container. Default is {@code null}. + * + * @param backPressureLimiter the backpressure limiter. + * @return this instance. + */ + B backPressureLimiter(BackPressureLimiter backPressureLimiter); + /** * Set the maximum interval between acknowledgements for batch acknowledgements. The default depends on the specific * {@link ContainerComponentFactory} implementation. diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java index 310b64519..e3d069bce 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java @@ -17,9 +17,11 @@ import java.time.Duration; import java.util.Arrays; +import java.util.Objects; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.util.Assert; @@ -35,33 +37,63 @@ public class SemaphoreBackPressureHandler implements BatchAwareBackPressureHandl private static final Logger logger = LoggerFactory.getLogger(SemaphoreBackPressureHandler.class); - private final Semaphore semaphore; + private final BackPressureLimiter backPressureLimiter; + + private final ReducibleSemaphore semaphore; private final int batchSize; + /** + * The theoretical maximum numbers of permits that can be acquired if no limit is set. + * @see #permitsLimit for the current limit. + */ private final int totalPermits; + /** + * The limit of permits that can be acquired at the current time. The permits limit is defined in the [0, + * totalPermits] interval. A value of {@literal 0} means that no permits can be acquired. + *

+ * This value is updated based on the downstream backpressure reported by the {@link #backPressureLimiter}. + */ + private final AtomicInteger permitsLimit; + + /** + * The duration to sleep when the queue processing is in standby. + */ + private final Duration standbyLimitPollingInterval; + private final Duration acquireTimeout; private final BackPressureMode backPressureConfiguration; private volatile CurrentThroughputMode currentThroughputMode; + /** + * The number of permits acquired in low throughput mode. This value is minimum value between {@link #permitsLimit} + * at the time of the acquire and {@link #totalPermits}. + */ + private final AtomicInteger lowThroughputAcquiredPermits = new AtomicInteger(0); + private final AtomicBoolean hasAcquiredFullPermits = new AtomicBoolean(false); private String id; + private final AtomicBoolean isDraining = new AtomicBoolean(false); + private SemaphoreBackPressureHandler(Builder builder) { this.batchSize = builder.batchSize; this.totalPermits = builder.totalPermits; + this.standbyLimitPollingInterval = builder.standbyLimitPollingInterval; this.acquireTimeout = builder.acquireTimeout; this.backPressureConfiguration = builder.backPressureMode; - this.semaphore = new Semaphore(totalPermits); + this.semaphore = new ReducibleSemaphore(totalPermits); this.currentThroughputMode = BackPressureMode.FIXED_HIGH_THROUGHPUT.equals(backPressureConfiguration) ? CurrentThroughputMode.HIGH : CurrentThroughputMode.LOW; logger.debug("SemaphoreBackPressureHandler created with configuration {} and {} total permits", backPressureConfiguration, totalPermits); + this.permitsLimit = new AtomicInteger(totalPermits); + this.backPressureLimiter = Objects.requireNonNullElse(builder.backPressureLimiter, () -> totalPermits); } public static Builder builder() { @@ -80,15 +112,17 @@ public String getId() { @Override public int request(int amount) throws InterruptedException { + updateAvailablePermitsBasedOnDownstreamBackpressure(); return tryAcquire(amount, this.currentThroughputMode) ? amount : 0; } // @formatter:off @Override public int requestBatch() throws InterruptedException { - return CurrentThroughputMode.LOW.equals(this.currentThroughputMode) - ? requestInLowThroughputMode() - : requestInHighThroughputMode(); + updateAvailablePermitsBasedOnDownstreamBackpressure(); + boolean useLowThroughput = CurrentThroughputMode.LOW.equals(this.currentThroughputMode) + || this.permitsLimit.get() < this.totalPermits; + return useLowThroughput ? requestInLowThroughputMode() : requestInHighThroughputMode(); } private int requestInHighThroughputMode() throws InterruptedException { @@ -103,10 +137,10 @@ private int tryAcquirePartial() throws InterruptedException { if (availablePermits == 0 || BackPressureMode.ALWAYS_POLL_MAX_MESSAGES.equals(this.backPressureConfiguration)) { return 0; } - int permitsToRequest = Math.min(availablePermits, this.batchSize); + int permitsToRequest = min(availablePermits, this.batchSize); CurrentThroughputMode currentThroughputModeNow = this.currentThroughputMode; - logger.trace("Trying to acquire partial batch of {} permits from {} available for {} in TM {}", - permitsToRequest, availablePermits, this.id, currentThroughputModeNow); + logger.trace("Trying to acquire partial batch of {} permits from {} (limit {}) available for {} in TM {}", + permitsToRequest, availablePermits, this.permitsLimit.get(), this.id, currentThroughputModeNow); boolean hasAcquiredPartial = tryAcquire(permitsToRequest, currentThroughputModeNow); return hasAcquiredPartial ? permitsToRequest : 0; } @@ -114,17 +148,35 @@ private int tryAcquirePartial() throws InterruptedException { private int requestInLowThroughputMode() throws InterruptedException { // Although LTM can be set / unset by many processes, only the MessageSource thread gets here, // so no actual concurrency - logger.debug("Trying to acquire full permits for {}. Permits left: {}", this.id, - this.semaphore.availablePermits()); - boolean hasAcquired = tryAcquire(this.totalPermits, CurrentThroughputMode.LOW); + logger.debug("Trying to acquire full permits for {}. Permits left: {}, Permits limit: {}", this.id, + this.semaphore.availablePermits(), this.permitsLimit.get()); + int permitsToRequest = min(this.permitsLimit.get(), this.totalPermits); + if (permitsToRequest == 0) { + logger.info("No permits usable for {} (limit = 0), sleeping for {}", this.id, + this.standbyLimitPollingInterval); + Thread.sleep(standbyLimitPollingInterval.toMillis()); + return 0; + } + boolean hasAcquired = tryAcquire(permitsToRequest, CurrentThroughputMode.LOW); if (hasAcquired) { - logger.debug("Acquired full permits for {}. Permits left: {}", this.id, this.semaphore.availablePermits()); + if (permitsToRequest >= this.totalPermits) { + logger.debug("Acquired full permits for {}. Permits left: {}, Permits limit: {}", this.id, + this.semaphore.availablePermits(), this.permitsLimit.get()); + } + else { + logger.debug("Acquired limited permits ({}) for {} . Permits left: {}, Permits limit: {}", + permitsToRequest, this.id, this.semaphore.availablePermits(), this.permitsLimit.get()); + } + int tokens = min(this.batchSize, permitsToRequest); // We've acquired all permits - there's no other process currently processing messages if (!this.hasAcquiredFullPermits.compareAndSet(false, true)) { - logger.warn("hasAcquiredFullPermits was already true. Permits left: {}", - this.semaphore.availablePermits()); + logger.warn("hasAcquiredFullPermits was already true. Permits left: {}, Permits limit: {}", + this.semaphore.availablePermits(), this.permitsLimit.get()); } - return this.batchSize; + else { + lowThroughputAcquiredPermits.set(permitsToRequest); + } + return tokens; } else { return 0; @@ -132,16 +184,20 @@ private int requestInLowThroughputMode() throws InterruptedException { } private boolean tryAcquire(int amount, CurrentThroughputMode currentThroughputModeNow) throws InterruptedException { + if (isDraining.get()) { + return false; + } logger.trace("Acquiring {} permits for {} in TM {}", amount, this.id, this.currentThroughputMode); boolean hasAcquired = this.semaphore.tryAcquire(amount, this.acquireTimeout.toMillis(), TimeUnit.MILLISECONDS); if (hasAcquired) { - logger.trace("{} permits acquired for {} in TM {}. Permits left: {}", amount, this.id, - currentThroughputModeNow, this.semaphore.availablePermits()); + logger.trace("{} permits acquired for {} in TM {}. Permits left: {}, Permits limit: {}", amount, this.id, + currentThroughputModeNow, this.semaphore.availablePermits(), this.permitsLimit.get()); } else { - logger.trace("Not able to acquire {} permits in {} milliseconds for {} in TM {}. Permits left: {}", amount, - this.acquireTimeout.toMillis(), this.id, currentThroughputModeNow, - this.semaphore.availablePermits()); + logger.trace( + "Not able to acquire {} permits in {} milliseconds for {} in TM {}. Permits left: {}, Permits limit: {}", + amount, this.acquireTimeout.toMillis(), this.id, currentThroughputModeNow, + this.semaphore.availablePermits(), this.permitsLimit.get()); } return hasAcquired; } @@ -181,11 +237,13 @@ public void release(int amount) { } private int getPermitsToRelease(int amount) { - return this.hasAcquiredFullPermits.compareAndSet(true, false) - // The first process that gets here should release all permits except for inflight messages - // We can have only one batch of messages at this point since we have all permits - ? this.totalPermits - (this.batchSize - amount) - : amount; + if (this.hasAcquiredFullPermits.compareAndSet(true, false)) { + int allAcquiredPermits = this.lowThroughputAcquiredPermits.getAndSet(0); + // The first process that gets here should release all permits except for inflight messages + // We can have only one batch of messages at this point since we have all permits + return (allAcquiredPermits - (min(this.batchSize, allAcquiredPermits) - amount)); + } + return amount; } private void maybeSwitchToHighThroughputMode(int amount) { @@ -200,6 +258,8 @@ private void maybeSwitchToHighThroughputMode(int amount) { public boolean drain(Duration timeout) { logger.debug("Waiting for up to {} seconds for approx. {} permits to be released for {}", timeout.getSeconds(), this.totalPermits - this.semaphore.availablePermits(), this.id); + isDraining.set(true); + updateMaxPermitsLimit(this.totalPermits); try { return this.semaphore.tryAcquire(this.totalPermits, (int) timeout.getSeconds(), TimeUnit.SECONDS); } @@ -209,6 +269,44 @@ public boolean drain(Duration timeout) { } } + private int min(int a, int p) { + return Math.max(0, Math.min(a, p)); + } + + private void updateAvailablePermitsBasedOnDownstreamBackpressure() { + if (!isDraining.get()) { + int limit = backPressureLimiter.limit(); + int newCurrentMaxPermits = min(limit, totalPermits); + updateMaxPermitsLimit(newCurrentMaxPermits); + if (isDraining.get()) { + updateMaxPermitsLimit(totalPermits); + } + } + } + + private void updateMaxPermitsLimit(int newCurrentMaxPermits) { + int oldValue = permitsLimit.getAndUpdate(i -> min(newCurrentMaxPermits, totalPermits)); + if (newCurrentMaxPermits < oldValue) { + int blockedPermits = oldValue - newCurrentMaxPermits; + semaphore.reducePermits(blockedPermits); + } + else if (newCurrentMaxPermits > oldValue) { + int releasedPermits = newCurrentMaxPermits - oldValue; + semaphore.release(releasedPermits); + } + } + + private static class ReducibleSemaphore extends Semaphore { + ReducibleSemaphore(int permits) { + super(permits); + } + + @Override + public void reducePermits(int reduction) { + super.reducePermits(reduction); + } + } + private enum CurrentThroughputMode { HIGH, @@ -223,10 +321,14 @@ public static class Builder { private int totalPermits; + private Duration standbyLimitPollingInterval; + private Duration acquireTimeout; private BackPressureMode backPressureMode; + private BackPressureLimiter backPressureLimiter; + public Builder batchSize(int batchSize) { this.batchSize = batchSize; return this; @@ -237,6 +339,11 @@ public Builder totalPermits(int totalPermits) { return this; } + public Builder standbyLimitPollingInterval(Duration standbyLimitPollingInterval) { + this.standbyLimitPollingInterval = standbyLimitPollingInterval; + return this; + } + public Builder acquireTimeout(Duration acquireTimeout) { this.acquireTimeout = acquireTimeout; return this; @@ -247,10 +354,14 @@ public Builder throughputConfiguration(BackPressureMode backPressureConfiguratio return this; } + public Builder backPressureLimiter(BackPressureLimiter backPressureLimiter) { + this.backPressureLimiter = backPressureLimiter; + return this; + } + public SemaphoreBackPressureHandler build() { - Assert.noNullElements( - Arrays.asList(this.batchSize, this.totalPermits, this.acquireTimeout, this.backPressureMode), - "Missing configuration"); + Assert.noNullElements(Arrays.asList(this.batchSize, this.totalPermits, this.standbyLimitPollingInterval, + this.acquireTimeout, this.backPressureMode), "Missing configuration"); return new SemaphoreBackPressureHandler(this); } diff --git a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java index 50bded839..b9834b338 100644 --- a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java +++ b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java @@ -27,6 +27,7 @@ import io.awspring.cloud.sqs.config.SqsBootstrapConfiguration; import io.awspring.cloud.sqs.config.SqsListenerConfigurer; import io.awspring.cloud.sqs.config.SqsMessageListenerContainerFactory; +import io.awspring.cloud.sqs.listener.BackPressureLimiter; import io.awspring.cloud.sqs.listener.BatchVisibility; import io.awspring.cloud.sqs.listener.ContainerComponentFactory; import io.awspring.cloud.sqs.listener.MessageListenerContainer; @@ -55,17 +56,24 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Random; import java.util.UUID; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.IntUnaryOperator; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -269,6 +277,7 @@ void manuallyCreatesInactiveContainer() throws Exception { logger.debug("Sent message to queue {} with messageBody {}", MANUALLY_CREATE_INACTIVE_CONTAINER_QUEUE_NAME, messageBody); assertThat(latchContainer.manuallyInactiveCreatedContainerLatch.await(10, TimeUnit.SECONDS)).isTrue(); + inactiveMessageListenerContainer.stop(); } // @formatter:off @@ -349,6 +358,298 @@ void maxConcurrentMessages() { assertDoesNotThrow(() -> latchContainer.maxConcurrentMessagesBarrier.await(10, TimeUnit.SECONDS)); } + static final class Limiter implements BackPressureLimiter { + private final AtomicInteger limit; + + Limiter(int max) { + limit = new AtomicInteger(max); + } + + public void setLimit(int value) { + logger.info("adjusting limit from {} to {}", limit.get(), value); + limit.set(value); + } + + @Override + public int limit() { + return Math.max(0, limit.get()); + } + } + + @ParameterizedTest + @CsvSource({ "2,2", "4,4", "5,5", "20,5" }) + void staticBackPressureLimitShouldCapQueueProcessingCapacity(int staticLimit, int expectedMaxConcurrentRequests) + throws Exception { + AtomicInteger concurrentRequest = new AtomicInteger(); + AtomicInteger maxConcurrentRequest = new AtomicInteger(); + Limiter limiter = new Limiter(staticLimit); + String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_STATIC_LIMIT_" + staticLimit; + IntStream.range(0, 10).forEach(index -> { + List> messages = create10Messages("staticBackPressureLimit" + staticLimit); + sqsTemplate.sendMany(queueName, messages); + }); + logger.debug("Sent 100 messages to queue {}", queueName); + var latch = new CountDownLatch(100); + var container = SqsMessageListenerContainer.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()) + .queueNames(queueName).configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .maxConcurrentMessages(5).maxMessagesPerPoll(5).backPressureLimiter(limiter)) + .messageListener(msg -> { + int concurrentRqs = concurrentRequest.incrementAndGet(); + maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); + sleep(50L); + logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), + maxConcurrentRequest.get(), latch.getCount()); + latch.countDown(); + concurrentRequest.decrementAndGet(); + }).build(); + container.start(); + assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); + assertThat(maxConcurrentRequest.get()).isEqualTo(expectedMaxConcurrentRequests); + container.stop(); + } + + @Test + void zeroBackPressureLimitShouldStopQueueProcessing() throws Exception { + AtomicInteger concurrentRequest = new AtomicInteger(); + AtomicInteger maxConcurrentRequest = new AtomicInteger(); + Limiter limiter = new Limiter(0); + String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_STATIC_LIMIT_0"; + IntStream.range(0, 10).forEach(index -> { + List> messages = create10Messages("staticBackPressureLimit0"); + sqsTemplate.sendMany(queueName, messages); + }); + logger.debug("Sent 100 messages to queue {}", queueName); + var latch = new CountDownLatch(100); + var container = SqsMessageListenerContainer.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()) + .queueNames(queueName).configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .maxConcurrentMessages(5).maxMessagesPerPoll(5).backPressureLimiter(limiter)) + .messageListener(msg -> { + int concurrentRqs = concurrentRequest.incrementAndGet(); + maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); + sleep(50L); + logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), + maxConcurrentRequest.get(), latch.getCount()); + latch.countDown(); + concurrentRequest.decrementAndGet(); + }).build(); + container.start(); + assertThat(latch.await(2, TimeUnit.SECONDS)).isFalse(); + assertThat(maxConcurrentRequest.get()).isZero(); + assertThat(latch.getCount()).isEqualTo(100L); + container.stop(); + } + + @Test + void changeInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Exception { + AtomicInteger concurrentRequest = new AtomicInteger(); + AtomicInteger maxConcurrentRequest = new AtomicInteger(); + Limiter limiter = new Limiter(5); + String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_SYNC_ADAPTIVE_LIMIT"; + int nbMessages = 280; + IntStream.range(0, nbMessages / 10).forEach(index -> { + List> messages = create10Messages("syncAdaptiveBackPressureLimit"); + sqsTemplate.sendMany(queueName, messages); + }); + logger.debug("Sent {} messages to queue {}", nbMessages, queueName); + var latch = new CountDownLatch(nbMessages); + var controlSemaphore = new Semaphore(0); + var advanceSemaphore = new Semaphore(0); + var container = SqsMessageListenerContainer.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()) + .queueNames(queueName).configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .maxConcurrentMessages(5).maxMessagesPerPoll(5).backPressureLimiter(limiter)) + .messageListener(msg -> { + try { + controlSemaphore.acquire(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + int concurrentRqs = concurrentRequest.incrementAndGet(); + maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); + latch.countDown(); + logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), + maxConcurrentRequest.get(), latch.getCount()); + sleep(10L); + concurrentRequest.decrementAndGet(); + advanceSemaphore.release(); + }).build(); + class Controller { + private final Semaphore advanceSemaphore; + private final Semaphore controlSemaphore; + private final Limiter limiter; + private final AtomicInteger maxConcurrentRequest; + + Controller(Semaphore advanceSemaphore, Semaphore controlSemaphore, Limiter limiter, + AtomicInteger maxConcurrentRequest) { + this.advanceSemaphore = advanceSemaphore; + this.controlSemaphore = controlSemaphore; + this.limiter = limiter; + this.maxConcurrentRequest = maxConcurrentRequest; + } + + public void updateLimit(int newLimit) { + limiter.setLimit(newLimit); + } + + void updateLimitAndWaitForReset(int newLimit) throws InterruptedException { + updateLimit(newLimit); + int atLeastTwoPollingCycles = 2 * 5; + controlSemaphore.release(atLeastTwoPollingCycles); + waitForAdvance(atLeastTwoPollingCycles); + maxConcurrentRequest.set(0); + } + + void advance(int permits) { + controlSemaphore.release(permits); + } + + void waitForAdvance(int permits) throws InterruptedException { + assertThat(advanceSemaphore.tryAcquire(permits, 5, TimeUnit.SECONDS)) + .withFailMessage(() -> "Waiting for %d permits timed out. Only %d permits available" + .formatted(permits, advanceSemaphore.availablePermits())) + .isTrue(); + } + } + var controller = new Controller(advanceSemaphore, controlSemaphore, limiter, maxConcurrentRequest); + try { + container.start(); + + controller.advance(50); + controller.waitForAdvance(50); + // not limiting queue processing capacity + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); + controller.updateLimitAndWaitForReset(2); + controller.advance(50); + + controller.waitForAdvance(50); + // limiting queue processing capacity + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(2); + controller.updateLimitAndWaitForReset(7); + controller.advance(50); + + controller.waitForAdvance(50); + // not limiting queue processing capacity + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); + controller.updateLimitAndWaitForReset(3); + controller.advance(50); + sleep(10L); + limiter.setLimit(1); + sleep(10L); + limiter.setLimit(2); + sleep(10L); + limiter.setLimit(3); + + controller.waitForAdvance(50); + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(3); + // stopping processing of the queue + controller.updateLimit(0); + controller.advance(50); + assertThat(advanceSemaphore.tryAcquire(10, 5, TimeUnit.SECONDS)) + .withFailMessage("Acquiring semaphore should have timed out as limit was set to 0").isFalse(); + + // resume queue processing + controller.updateLimit(6); + + controller.waitForAdvance(50); + assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); + } + finally { + container.stop(); + } + } + + /** + * This test simulates a progressive change in the back pressure limit. Unlike + * {@link #changeInBackPressureLimitShouldAdaptQueueProcessingCapacity()}, this test does not block message + * consumption while updating the limit. + *

+ * The limit is updated in a loop until all messages are consumed. The update follows a triangle wave pattern with a + * minimum of 0, a maximum of 15, and a period of 30 iterations. After each update of the limit, the test waits up + * to 10ms and samples the maximum number of concurrent messages that were processed since the update. This number + * can be higher than the defined limit during the adaptation period of the decreasing limit wave. For the + * increasing limit wave, it is usually lower due to the adaptation delay. In both cases, the maximum number of + * concurrent messages being processed rapidly converges toward the defined limit. + *

+ * The test passes if the sum of the sampled maximum number of concurrently processed messages is lower than the sum + * of the limits at those points in time. + */ + @Test + void unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Exception { + AtomicInteger concurrentRequest = new AtomicInteger(); + AtomicInteger maxConcurrentRequest = new AtomicInteger(); + Limiter limiter = new Limiter(0); + String queueName = "REACTIVE_BACK_PRESSURE_LIMITER_QUEUE_NAME_ADAPTIVE_LIMIT"; + int nbMessages = 1000; + Semaphore advanceSemaphore = new Semaphore(0); + IntStream.range(0, nbMessages / 10).forEach(index -> { + List> messages = create10Messages("reactAdaptiveBackPressureLimit"); + sqsTemplate.sendMany(queueName, messages); + }); + logger.debug("Sent {} messages to queue {}", nbMessages, queueName); + var latch = new CountDownLatch(nbMessages); + var container = SqsMessageListenerContainer.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()) + .queueNames(queueName) + .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .standbyLimitPollingInterval(Duration.ofMillis(1)).maxConcurrentMessages(10) + .maxMessagesPerPoll(10).backPressureLimiter(limiter)) + .messageListener(msg -> { + int currentConcurrentRq = concurrentRequest.incrementAndGet(); + maxConcurrentRequest.updateAndGet(max -> Math.max(max, currentConcurrentRq)); + sleep(ThreadLocalRandom.current().nextInt(10)); + latch.countDown(); + logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), + maxConcurrentRequest.get(), latch.getCount()); + concurrentRequest.decrementAndGet(); + advanceSemaphore.release(); + }).build(); + IntUnaryOperator progressiveLimitChange = (int x) -> { + int period = 30; + int halfPeriod = period / 2; + if (x % period < halfPeriod) { + return (x % halfPeriod); + } + else { + return (halfPeriod - (x % halfPeriod)); + } + }; + try { + container.start(); + Random random = new Random(); + int limitsSum = 0; + int maxConcurrentRqSum = 0; + int changeLimitCount = 0; + while (latch.getCount() > 0 && changeLimitCount < nbMessages) { + changeLimitCount++; + int limit = progressiveLimitChange.applyAsInt(changeLimitCount); + limiter.setLimit(limit); + maxConcurrentRequest.set(0); + sleep(random.nextInt(10)); + int actualLimit = Math.min(10, limit); + int max = maxConcurrentRequest.getAndSet(0); + if (max > 0) { + // Ignore iterations where nothing was polled (messages consumption slower than iteration) + limitsSum += actualLimit; + maxConcurrentRqSum += max; + } + } + assertThat(maxConcurrentRqSum).isLessThanOrEqualTo(limitsSum); + assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); + } + finally { + container.stop(); + } + } + + private static void sleep(long millis) { + try { + Thread.sleep(millis); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + static class ReceivesMessageListener { @Autowired From 93cb447c3e66a6c22e59d6958fc4b820f5eb9acc Mon Sep 17 00:00:00 2001 From: Loic Rouchon Date: Thu, 2 Jan 2025 17:18:24 +0100 Subject: [PATCH 2/9] Use a wrapper approach for dynamically limit the permits of SemaphoreBackPressureHandler (#1251) --- ...tractPipelineMessageListenerContainer.java | 13 +- .../listener/BackPressureHandlerLimiter.java | 153 ++++++++++++++++ .../SemaphoreBackPressureHandler.java | 167 +++--------------- 3 files changed, 190 insertions(+), 143 deletions(-) create mode 100644 spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java index 79073c96c..e11a92bdb 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java @@ -226,12 +226,17 @@ private TaskExecutor validateCustomExecutor(TaskExecutor taskExecutor) { protected BackPressureHandler createBackPressureHandler() { O containerOptions = getContainerOptions(); - return SemaphoreBackPressureHandler.builder().batchSize(containerOptions.getMaxMessagesPerPoll()) + BatchAwareBackPressureHandler backPressureHandler = SemaphoreBackPressureHandler.builder() + .batchSize(containerOptions.getMaxMessagesPerPoll()) .totalPermits(containerOptions.getMaxConcurrentMessages()) - .standbyLimitPollingInterval(containerOptions.getStandbyLimitPollingInterval()) .acquireTimeout(containerOptions.getMaxDelayBetweenPolls()) - .throughputConfiguration(containerOptions.getBackPressureMode()) - .backPressureLimiter(containerOptions.getBackPressureLimiter()).build(); + .throughputConfiguration(containerOptions.getBackPressureMode()).build(); + if (containerOptions.getBackPressureLimiter() != null) { + backPressureHandler = new BackPressureHandlerLimiter(backPressureHandler, + containerOptions.getBackPressureLimiter(), containerOptions.getStandbyLimitPollingInterval(), + containerOptions.getMaxDelayBetweenPolls()); + } + return backPressureHandler; } protected TaskExecutor createSourcesTaskExecutor() { diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java new file mode 100644 index 000000000..aeb5a61cb --- /dev/null +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java @@ -0,0 +1,153 @@ +/* + * Copyright 2013-2025 the original author or authors. + * + * Licensed 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 + * + * https://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 io.awspring.cloud.sqs.listener; + +import java.time.Duration; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A {@link BatchAwareBackPressureHandler} implementation that uses an internal {@link Semaphore} for adapting the + * maximum number of permits that can be acquired by the {@link #backPressureHandler} based on the downstream + * backpressure limit computed by the {@link #backPressureLimiter}. + * + * @see BackPressureLimiter + */ +public class BackPressureHandlerLimiter implements BatchAwareBackPressureHandler { + + /** + * The {@link BatchAwareBackPressureHandler} which permits should be limited by the {@link #backPressureLimiter}. + */ + private final BatchAwareBackPressureHandler backPressureHandler; + + /** + * The {@link BackPressureLimiter} which computes a limit on how many permits can be requested at a given moment. + */ + private final BackPressureLimiter backPressureLimiter; + + /** + * The duration to wait for permits to be acquired. + */ + private final Duration acquireTimeout; + + /** + * The duration to sleep when the queue processing is in standby. + */ + private final Duration standbyLimitPollingInterval; + + /** + * The limit of permits that can be acquired at the current time. The permits limit is defined in the [0, + * Integer.MAX_VALUE] interval. A value of {@literal 0} means that no permits can be acquired. + *

+ * This value is updated based on the downstream backpressure reported by the {@link #backPressureLimiter}. + */ + private final AtomicInteger permitsLimit = new AtomicInteger(0); + + private final ReducibleSemaphore semaphore = new ReducibleSemaphore(0); + + public BackPressureHandlerLimiter(BatchAwareBackPressureHandler backPressureHandler, + BackPressureLimiter backPressureLimiter, Duration standbyLimitPollingInterval, Duration acquireTimeout) { + this.backPressureHandler = backPressureHandler; + this.backPressureLimiter = backPressureLimiter; + this.acquireTimeout = acquireTimeout; + this.standbyLimitPollingInterval = standbyLimitPollingInterval; + } + + @Override + public int requestBatch() throws InterruptedException { + int permits = updatePermitsLimit(); + int batchSize = getBatchSize(); + if (permits < batchSize) { + return acquirePermits(permits, backPressureHandler::request); + } + return acquirePermits(batchSize, p -> backPressureHandler.requestBatch()); + } + + @Override + public void releaseBatch() { + semaphore.release(getBatchSize()); + backPressureHandler.releaseBatch(); + } + + @Override + public int getBatchSize() { + return backPressureHandler.getBatchSize(); + } + + @Override + public int request(int amount) throws InterruptedException { + int permits = Math.min(updatePermitsLimit(), amount); + return acquirePermits(permits, backPressureHandler::request); + } + + @Override + public void release(int amount) { + semaphore.release(amount); + backPressureHandler.release(amount); + } + + @Override + public boolean drain(Duration timeout) { + return backPressureHandler.drain(timeout); + } + + private int updatePermitsLimit() { + return permitsLimit.updateAndGet(oldLimit -> { + int newLimit = Math.max(0, backPressureLimiter.limit()); + if (newLimit < oldLimit) { + int blockedPermits = oldLimit - newLimit; + semaphore.reducePermits(blockedPermits); + } + else if (newLimit > oldLimit) { + int releasedPermits = newLimit - oldLimit; + semaphore.release(releasedPermits); + } + return newLimit; + }); + } + + private interface PermitsRequester { + int request(int amount) throws InterruptedException; + } + + private int acquirePermits(int amount, PermitsRequester permitsRequester) throws InterruptedException { + if (amount == 0) { + Thread.sleep(standbyLimitPollingInterval.toMillis()); + return 0; + } + if (semaphore.tryAcquire(amount, acquireTimeout.toMillis(), TimeUnit.MILLISECONDS)) { + int obtained = permitsRequester.request(amount); + if (obtained < amount) { + semaphore.release(amount - obtained); + } + return obtained; + } + return 0; + } + + private static class ReducibleSemaphore extends Semaphore { + + ReducibleSemaphore(int permits) { + super(permits); + } + + @Override + public void reducePermits(int reduction) { + super.reducePermits(reduction); + } + } +} diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java index e3d069bce..310b64519 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java @@ -17,11 +17,9 @@ import java.time.Duration; import java.util.Arrays; -import java.util.Objects; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.util.Assert; @@ -37,63 +35,33 @@ public class SemaphoreBackPressureHandler implements BatchAwareBackPressureHandl private static final Logger logger = LoggerFactory.getLogger(SemaphoreBackPressureHandler.class); - private final BackPressureLimiter backPressureLimiter; - - private final ReducibleSemaphore semaphore; + private final Semaphore semaphore; private final int batchSize; - /** - * The theoretical maximum numbers of permits that can be acquired if no limit is set. - * @see #permitsLimit for the current limit. - */ private final int totalPermits; - /** - * The limit of permits that can be acquired at the current time. The permits limit is defined in the [0, - * totalPermits] interval. A value of {@literal 0} means that no permits can be acquired. - *

- * This value is updated based on the downstream backpressure reported by the {@link #backPressureLimiter}. - */ - private final AtomicInteger permitsLimit; - - /** - * The duration to sleep when the queue processing is in standby. - */ - private final Duration standbyLimitPollingInterval; - private final Duration acquireTimeout; private final BackPressureMode backPressureConfiguration; private volatile CurrentThroughputMode currentThroughputMode; - /** - * The number of permits acquired in low throughput mode. This value is minimum value between {@link #permitsLimit} - * at the time of the acquire and {@link #totalPermits}. - */ - private final AtomicInteger lowThroughputAcquiredPermits = new AtomicInteger(0); - private final AtomicBoolean hasAcquiredFullPermits = new AtomicBoolean(false); private String id; - private final AtomicBoolean isDraining = new AtomicBoolean(false); - private SemaphoreBackPressureHandler(Builder builder) { this.batchSize = builder.batchSize; this.totalPermits = builder.totalPermits; - this.standbyLimitPollingInterval = builder.standbyLimitPollingInterval; this.acquireTimeout = builder.acquireTimeout; this.backPressureConfiguration = builder.backPressureMode; - this.semaphore = new ReducibleSemaphore(totalPermits); + this.semaphore = new Semaphore(totalPermits); this.currentThroughputMode = BackPressureMode.FIXED_HIGH_THROUGHPUT.equals(backPressureConfiguration) ? CurrentThroughputMode.HIGH : CurrentThroughputMode.LOW; logger.debug("SemaphoreBackPressureHandler created with configuration {} and {} total permits", backPressureConfiguration, totalPermits); - this.permitsLimit = new AtomicInteger(totalPermits); - this.backPressureLimiter = Objects.requireNonNullElse(builder.backPressureLimiter, () -> totalPermits); } public static Builder builder() { @@ -112,17 +80,15 @@ public String getId() { @Override public int request(int amount) throws InterruptedException { - updateAvailablePermitsBasedOnDownstreamBackpressure(); return tryAcquire(amount, this.currentThroughputMode) ? amount : 0; } // @formatter:off @Override public int requestBatch() throws InterruptedException { - updateAvailablePermitsBasedOnDownstreamBackpressure(); - boolean useLowThroughput = CurrentThroughputMode.LOW.equals(this.currentThroughputMode) - || this.permitsLimit.get() < this.totalPermits; - return useLowThroughput ? requestInLowThroughputMode() : requestInHighThroughputMode(); + return CurrentThroughputMode.LOW.equals(this.currentThroughputMode) + ? requestInLowThroughputMode() + : requestInHighThroughputMode(); } private int requestInHighThroughputMode() throws InterruptedException { @@ -137,10 +103,10 @@ private int tryAcquirePartial() throws InterruptedException { if (availablePermits == 0 || BackPressureMode.ALWAYS_POLL_MAX_MESSAGES.equals(this.backPressureConfiguration)) { return 0; } - int permitsToRequest = min(availablePermits, this.batchSize); + int permitsToRequest = Math.min(availablePermits, this.batchSize); CurrentThroughputMode currentThroughputModeNow = this.currentThroughputMode; - logger.trace("Trying to acquire partial batch of {} permits from {} (limit {}) available for {} in TM {}", - permitsToRequest, availablePermits, this.permitsLimit.get(), this.id, currentThroughputModeNow); + logger.trace("Trying to acquire partial batch of {} permits from {} available for {} in TM {}", + permitsToRequest, availablePermits, this.id, currentThroughputModeNow); boolean hasAcquiredPartial = tryAcquire(permitsToRequest, currentThroughputModeNow); return hasAcquiredPartial ? permitsToRequest : 0; } @@ -148,35 +114,17 @@ private int tryAcquirePartial() throws InterruptedException { private int requestInLowThroughputMode() throws InterruptedException { // Although LTM can be set / unset by many processes, only the MessageSource thread gets here, // so no actual concurrency - logger.debug("Trying to acquire full permits for {}. Permits left: {}, Permits limit: {}", this.id, - this.semaphore.availablePermits(), this.permitsLimit.get()); - int permitsToRequest = min(this.permitsLimit.get(), this.totalPermits); - if (permitsToRequest == 0) { - logger.info("No permits usable for {} (limit = 0), sleeping for {}", this.id, - this.standbyLimitPollingInterval); - Thread.sleep(standbyLimitPollingInterval.toMillis()); - return 0; - } - boolean hasAcquired = tryAcquire(permitsToRequest, CurrentThroughputMode.LOW); + logger.debug("Trying to acquire full permits for {}. Permits left: {}", this.id, + this.semaphore.availablePermits()); + boolean hasAcquired = tryAcquire(this.totalPermits, CurrentThroughputMode.LOW); if (hasAcquired) { - if (permitsToRequest >= this.totalPermits) { - logger.debug("Acquired full permits for {}. Permits left: {}, Permits limit: {}", this.id, - this.semaphore.availablePermits(), this.permitsLimit.get()); - } - else { - logger.debug("Acquired limited permits ({}) for {} . Permits left: {}, Permits limit: {}", - permitsToRequest, this.id, this.semaphore.availablePermits(), this.permitsLimit.get()); - } - int tokens = min(this.batchSize, permitsToRequest); + logger.debug("Acquired full permits for {}. Permits left: {}", this.id, this.semaphore.availablePermits()); // We've acquired all permits - there's no other process currently processing messages if (!this.hasAcquiredFullPermits.compareAndSet(false, true)) { - logger.warn("hasAcquiredFullPermits was already true. Permits left: {}, Permits limit: {}", - this.semaphore.availablePermits(), this.permitsLimit.get()); + logger.warn("hasAcquiredFullPermits was already true. Permits left: {}", + this.semaphore.availablePermits()); } - else { - lowThroughputAcquiredPermits.set(permitsToRequest); - } - return tokens; + return this.batchSize; } else { return 0; @@ -184,20 +132,16 @@ private int requestInLowThroughputMode() throws InterruptedException { } private boolean tryAcquire(int amount, CurrentThroughputMode currentThroughputModeNow) throws InterruptedException { - if (isDraining.get()) { - return false; - } logger.trace("Acquiring {} permits for {} in TM {}", amount, this.id, this.currentThroughputMode); boolean hasAcquired = this.semaphore.tryAcquire(amount, this.acquireTimeout.toMillis(), TimeUnit.MILLISECONDS); if (hasAcquired) { - logger.trace("{} permits acquired for {} in TM {}. Permits left: {}, Permits limit: {}", amount, this.id, - currentThroughputModeNow, this.semaphore.availablePermits(), this.permitsLimit.get()); + logger.trace("{} permits acquired for {} in TM {}. Permits left: {}", amount, this.id, + currentThroughputModeNow, this.semaphore.availablePermits()); } else { - logger.trace( - "Not able to acquire {} permits in {} milliseconds for {} in TM {}. Permits left: {}, Permits limit: {}", - amount, this.acquireTimeout.toMillis(), this.id, currentThroughputModeNow, - this.semaphore.availablePermits(), this.permitsLimit.get()); + logger.trace("Not able to acquire {} permits in {} milliseconds for {} in TM {}. Permits left: {}", amount, + this.acquireTimeout.toMillis(), this.id, currentThroughputModeNow, + this.semaphore.availablePermits()); } return hasAcquired; } @@ -237,13 +181,11 @@ public void release(int amount) { } private int getPermitsToRelease(int amount) { - if (this.hasAcquiredFullPermits.compareAndSet(true, false)) { - int allAcquiredPermits = this.lowThroughputAcquiredPermits.getAndSet(0); - // The first process that gets here should release all permits except for inflight messages - // We can have only one batch of messages at this point since we have all permits - return (allAcquiredPermits - (min(this.batchSize, allAcquiredPermits) - amount)); - } - return amount; + return this.hasAcquiredFullPermits.compareAndSet(true, false) + // The first process that gets here should release all permits except for inflight messages + // We can have only one batch of messages at this point since we have all permits + ? this.totalPermits - (this.batchSize - amount) + : amount; } private void maybeSwitchToHighThroughputMode(int amount) { @@ -258,8 +200,6 @@ private void maybeSwitchToHighThroughputMode(int amount) { public boolean drain(Duration timeout) { logger.debug("Waiting for up to {} seconds for approx. {} permits to be released for {}", timeout.getSeconds(), this.totalPermits - this.semaphore.availablePermits(), this.id); - isDraining.set(true); - updateMaxPermitsLimit(this.totalPermits); try { return this.semaphore.tryAcquire(this.totalPermits, (int) timeout.getSeconds(), TimeUnit.SECONDS); } @@ -269,44 +209,6 @@ public boolean drain(Duration timeout) { } } - private int min(int a, int p) { - return Math.max(0, Math.min(a, p)); - } - - private void updateAvailablePermitsBasedOnDownstreamBackpressure() { - if (!isDraining.get()) { - int limit = backPressureLimiter.limit(); - int newCurrentMaxPermits = min(limit, totalPermits); - updateMaxPermitsLimit(newCurrentMaxPermits); - if (isDraining.get()) { - updateMaxPermitsLimit(totalPermits); - } - } - } - - private void updateMaxPermitsLimit(int newCurrentMaxPermits) { - int oldValue = permitsLimit.getAndUpdate(i -> min(newCurrentMaxPermits, totalPermits)); - if (newCurrentMaxPermits < oldValue) { - int blockedPermits = oldValue - newCurrentMaxPermits; - semaphore.reducePermits(blockedPermits); - } - else if (newCurrentMaxPermits > oldValue) { - int releasedPermits = newCurrentMaxPermits - oldValue; - semaphore.release(releasedPermits); - } - } - - private static class ReducibleSemaphore extends Semaphore { - ReducibleSemaphore(int permits) { - super(permits); - } - - @Override - public void reducePermits(int reduction) { - super.reducePermits(reduction); - } - } - private enum CurrentThroughputMode { HIGH, @@ -321,14 +223,10 @@ public static class Builder { private int totalPermits; - private Duration standbyLimitPollingInterval; - private Duration acquireTimeout; private BackPressureMode backPressureMode; - private BackPressureLimiter backPressureLimiter; - public Builder batchSize(int batchSize) { this.batchSize = batchSize; return this; @@ -339,11 +237,6 @@ public Builder totalPermits(int totalPermits) { return this; } - public Builder standbyLimitPollingInterval(Duration standbyLimitPollingInterval) { - this.standbyLimitPollingInterval = standbyLimitPollingInterval; - return this; - } - public Builder acquireTimeout(Duration acquireTimeout) { this.acquireTimeout = acquireTimeout; return this; @@ -354,14 +247,10 @@ public Builder throughputConfiguration(BackPressureMode backPressureConfiguratio return this; } - public Builder backPressureLimiter(BackPressureLimiter backPressureLimiter) { - this.backPressureLimiter = backPressureLimiter; - return this; - } - public SemaphoreBackPressureHandler build() { - Assert.noNullElements(Arrays.asList(this.batchSize, this.totalPermits, this.standbyLimitPollingInterval, - this.acquireTimeout, this.backPressureMode), "Missing configuration"); + Assert.noNullElements( + Arrays.asList(this.batchSize, this.totalPermits, this.acquireTimeout, this.backPressureMode), + "Missing configuration"); return new SemaphoreBackPressureHandler(this); } From 432d49025a953b16393180ab55b12b2dc5f60857 Mon Sep 17 00:00:00 2001 From: Loic Rouchon Date: Fri, 3 Jan 2025 14:14:14 +0100 Subject: [PATCH 3/9] Introduce a CompositeBackPressureHandler allowing for composition of BackPressureHandlers (#1251) --- ...tractPipelineMessageListenerContainer.java | 19 +++-- .../sqs/listener/BackPressureHandler.java | 41 ++++++++- .../listener/BackPressureHandlerLimiter.java | 68 ++++++--------- .../BatchAwareBackPressureHandler.java | 14 ---- .../CompositeBackPressureHandler.java | 84 +++++++++++++++++++ .../SemaphoreBackPressureHandler.java | 81 ++++++++---------- .../source/AbstractPollingMessageSource.java | 20 ++--- 7 files changed, 201 insertions(+), 126 deletions(-) create mode 100644 spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java index e11a92bdb..401f07248 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java @@ -35,6 +35,7 @@ import io.awspring.cloud.sqs.listener.source.AcknowledgementProcessingMessageSource; import io.awspring.cloud.sqs.listener.source.MessageSource; import io.awspring.cloud.sqs.listener.source.PollingMessageSource; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -226,17 +227,17 @@ private TaskExecutor validateCustomExecutor(TaskExecutor taskExecutor) { protected BackPressureHandler createBackPressureHandler() { O containerOptions = getContainerOptions(); - BatchAwareBackPressureHandler backPressureHandler = SemaphoreBackPressureHandler.builder() - .batchSize(containerOptions.getMaxMessagesPerPoll()) - .totalPermits(containerOptions.getMaxConcurrentMessages()) - .acquireTimeout(containerOptions.getMaxDelayBetweenPolls()) - .throughputConfiguration(containerOptions.getBackPressureMode()).build(); + List backPressureHandlers = new ArrayList<>(2); + Duration acquireTimeout = containerOptions.getMaxDelayBetweenPolls(); + int batchSize = containerOptions.getMaxMessagesPerPoll(); + backPressureHandlers.add(SemaphoreBackPressureHandler.builder().batchSize(batchSize) + .totalPermits(containerOptions.getMaxConcurrentMessages()).acquireTimeout(acquireTimeout) + .throughputConfiguration(containerOptions.getBackPressureMode()).build()); if (containerOptions.getBackPressureLimiter() != null) { - backPressureHandler = new BackPressureHandlerLimiter(backPressureHandler, - containerOptions.getBackPressureLimiter(), containerOptions.getStandbyLimitPollingInterval(), - containerOptions.getMaxDelayBetweenPolls()); + backPressureHandlers.add(new BackPressureHandlerLimiter(containerOptions.getBackPressureLimiter(), + acquireTimeout, containerOptions.getStandbyLimitPollingInterval(), batchSize)); } - return backPressureHandler; + return new CompositeBackPressureHandler(backPressureHandlers, batchSize); } protected TaskExecutor createSourcesTaskExecutor() { diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java index 1d76d6589..f2ff274b1 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java @@ -29,7 +29,7 @@ public interface BackPressureHandler { /** - * Request a number of permits. Each obtained permit allows the + * Requests a number of permits. Each obtained permit allows the * {@link io.awspring.cloud.sqs.listener.source.MessageSource} to retrieve one message. * @param amount the amount of permits to request. * @return the amount of permits obtained. @@ -38,11 +38,24 @@ public interface BackPressureHandler { int request(int amount) throws InterruptedException; /** - * Release the specified amount of permits. Each message that has been processed should release one permit, whether - * processing was successful or not. + * Releases the specified amount of permits for processed messages. Each message that has been processed should + * release one permit, whether processing was successful or not. + *

+ * This method can is called in the following use cases: + *

    + *
  • {@link ReleaseReason#LIMITED}: permits were not used because another BackPressureHandler has a lower permits + * limit and the difference in permits needs to be returned.
  • + *
  • {@link ReleaseReason#NONE_FETCHED}: none of the permits were actually used because no messages were retrieved + * from SQS. Permits need to be returned.
  • + *
  • {@link ReleaseReason#PARTIAL_FETCH}: some of the permits were used (some messages were retrieved from SQS). + * The unused ones need to be returned. The amount to be returned might be {@literal 0}, in which case it means all + * the permits will be used as the same number of messages were fetched from SQS.
  • + *
  • {@link ReleaseReason#PROCESSED}: a message processing finished, successfully or not.
  • + *
* @param amount the amount of permits to release. + * @param reason the reason why the permits were released. */ - void release(int amount); + void release(int amount, ReleaseReason reason); /** * Attempts to acquire all permits up to the specified timeout. If successful, means all permits were returned and @@ -52,4 +65,24 @@ public interface BackPressureHandler { */ boolean drain(Duration timeout); + enum ReleaseReason { + /** + * Permits were not used because another BackPressureHandler has a lower permits limit and the difference need + * to be aligned across all handlers. + */ + LIMITED, + /** + * No messages were retrieved from SQS, so all permits need to be returned. + */ + NONE_FETCHED, + /** + * Some messages were fetched from SQS. Unused permits need to be returned. + */ + PARTIAL_FETCH, + /** + * The processing of one or more messages finished, successfully or not. + */ + PROCESSED; + } + } diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java index aeb5a61cb..cd031a129 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java @@ -27,12 +27,7 @@ * * @see BackPressureLimiter */ -public class BackPressureHandlerLimiter implements BatchAwareBackPressureHandler { - - /** - * The {@link BatchAwareBackPressureHandler} which permits should be limited by the {@link #backPressureLimiter}. - */ - private final BatchAwareBackPressureHandler backPressureHandler; +public class BackPressureHandlerLimiter implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { /** * The {@link BackPressureLimiter} which computes a limit on how many permits can be requested at a given moment. @@ -59,50 +54,54 @@ public class BackPressureHandlerLimiter implements BatchAwareBackPressureHandler private final ReducibleSemaphore semaphore = new ReducibleSemaphore(0); - public BackPressureHandlerLimiter(BatchAwareBackPressureHandler backPressureHandler, - BackPressureLimiter backPressureLimiter, Duration standbyLimitPollingInterval, Duration acquireTimeout) { - this.backPressureHandler = backPressureHandler; + private final int batchSize; + + private String id; + + public BackPressureHandlerLimiter(BackPressureLimiter backPressureLimiter, Duration acquireTimeout, + Duration standbyLimitPollingInterval, int batchSize) { this.backPressureLimiter = backPressureLimiter; this.acquireTimeout = acquireTimeout; this.standbyLimitPollingInterval = standbyLimitPollingInterval; + this.batchSize = batchSize; } @Override - public int requestBatch() throws InterruptedException { - int permits = updatePermitsLimit(); - int batchSize = getBatchSize(); - if (permits < batchSize) { - return acquirePermits(permits, backPressureHandler::request); - } - return acquirePermits(batchSize, p -> backPressureHandler.requestBatch()); + public void setId(String id) { + this.id = id; } @Override - public void releaseBatch() { - semaphore.release(getBatchSize()); - backPressureHandler.releaseBatch(); + public String getId() { + return id; } @Override - public int getBatchSize() { - return backPressureHandler.getBatchSize(); + public int requestBatch() throws InterruptedException { + return request(batchSize); } @Override public int request(int amount) throws InterruptedException { int permits = Math.min(updatePermitsLimit(), amount); - return acquirePermits(permits, backPressureHandler::request); + if (permits == 0) { + Thread.sleep(standbyLimitPollingInterval.toMillis()); + return 0; + } + if (semaphore.tryAcquire(permits, acquireTimeout.toMillis(), TimeUnit.MILLISECONDS)) { + return permits; + } + return 0; } @Override - public void release(int amount) { + public void release(int amount, ReleaseReason reason) { semaphore.release(amount); - backPressureHandler.release(amount); } @Override public boolean drain(Duration timeout) { - return backPressureHandler.drain(timeout); + return true; } private int updatePermitsLimit() { @@ -120,25 +119,6 @@ else if (newLimit > oldLimit) { }); } - private interface PermitsRequester { - int request(int amount) throws InterruptedException; - } - - private int acquirePermits(int amount, PermitsRequester permitsRequester) throws InterruptedException { - if (amount == 0) { - Thread.sleep(standbyLimitPollingInterval.toMillis()); - return 0; - } - if (semaphore.tryAcquire(amount, acquireTimeout.toMillis(), TimeUnit.MILLISECONDS)) { - int obtained = permitsRequester.request(amount); - if (obtained < amount) { - semaphore.release(amount - obtained); - } - return obtained; - } - return 0; - } - private static class ReducibleSemaphore extends Semaphore { ReducibleSemaphore(int permits) { diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java index 51e12e0a0..c9ce20f9b 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java @@ -30,18 +30,4 @@ public interface BatchAwareBackPressureHandler extends BackPressureHandler { * @throws InterruptedException if the Thread is interrupted while waiting for permits. */ int requestBatch() throws InterruptedException; - - /** - * Release a batch of permits. This has the semantics of letting the {@link BackPressureHandler} know that all - * permits from a batch are being released, in opposition to {@link #release(int)} in which any number of permits - * can be specified. - */ - void releaseBatch(); - - /** - * Return the configured batch size for this handler. - * @return the batch size. - */ - int getBatchSize(); - } diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java new file mode 100644 index 000000000..42202438b --- /dev/null +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java @@ -0,0 +1,84 @@ +/* + * Copyright 2013-2025 the original author or authors. + * + * Licensed 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 + * + * https://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 io.awspring.cloud.sqs.listener; + +import java.time.Duration; +import java.util.List; + +public class CompositeBackPressureHandler implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { + + private final List backPressureHandlers; + + private final int batchSize; + + private String id; + + public CompositeBackPressureHandler(List backPressureHandlers, int batchSize) { + this.backPressureHandlers = backPressureHandlers; + this.batchSize = batchSize; + } + + @Override + public void setId(String id) { + this.id = id; + backPressureHandlers.stream().filter(IdentifiableContainerComponent.class::isInstance) + .map(IdentifiableContainerComponent.class::cast) + .forEach(bph -> bph.setId(bph.getClass().getSimpleName() + "-" + id)); + } + + @Override + public String getId() { + return id; + } + + @Override + public int requestBatch() throws InterruptedException { + return request(batchSize); + } + + @Override + public int request(int amount) throws InterruptedException { + int obtained = amount; + int[] obtainedPerBph = new int[backPressureHandlers.size()]; + for (int i = 0; i < backPressureHandlers.size() && obtained > 0; i++) { + obtainedPerBph[i] = backPressureHandlers.get(i).request(obtained); + obtained = Math.min(obtained, obtainedPerBph[i]); + } + for (int i = 0; i < backPressureHandlers.size(); i++) { + int obtainedForBph = obtainedPerBph[i]; + if (obtainedForBph > obtained) { + backPressureHandlers.get(i).release(obtainedForBph - obtained, ReleaseReason.LIMITED); + } + } + return obtained; + } + + @Override + public void release(int amount, ReleaseReason reason) { + for (BackPressureHandler handler : backPressureHandlers) { + handler.release(amount, reason); + } + } + + @Override + public boolean drain(Duration timeout) { + boolean result = true; + for (BackPressureHandler handler : backPressureHandlers) { + result &= !handler.drain(timeout); + } + return result; + } +} diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java index 310b64519..70ed3f306 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java @@ -19,7 +19,7 @@ import java.util.Arrays; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.util.Assert; @@ -47,7 +47,7 @@ public class SemaphoreBackPressureHandler implements BatchAwareBackPressureHandl private volatile CurrentThroughputMode currentThroughputMode; - private final AtomicBoolean hasAcquiredFullPermits = new AtomicBoolean(false); + private final AtomicInteger lowThroughputPermitsAcquired = new AtomicInteger(0); private String id; @@ -79,31 +79,31 @@ public String getId() { } @Override - public int request(int amount) throws InterruptedException { - return tryAcquire(amount, this.currentThroughputMode) ? amount : 0; + public int requestBatch() throws InterruptedException { + return request(batchSize); } // @formatter:off @Override - public int requestBatch() throws InterruptedException { + public int request(int amount) throws InterruptedException { return CurrentThroughputMode.LOW.equals(this.currentThroughputMode) - ? requestInLowThroughputMode() - : requestInHighThroughputMode(); + ? requestInLowThroughputMode(amount) + : requestInHighThroughputMode(amount); } - private int requestInHighThroughputMode() throws InterruptedException { - return tryAcquire(this.batchSize, CurrentThroughputMode.HIGH) - ? this.batchSize - : tryAcquirePartial(); + private int requestInHighThroughputMode(int amount) throws InterruptedException { + return tryAcquire(amount, CurrentThroughputMode.HIGH) + ? amount + : tryAcquirePartial(amount); } // @formatter:on - private int tryAcquirePartial() throws InterruptedException { + private int tryAcquirePartial(int max) throws InterruptedException { int availablePermits = this.semaphore.availablePermits(); if (availablePermits == 0 || BackPressureMode.ALWAYS_POLL_MAX_MESSAGES.equals(this.backPressureConfiguration)) { return 0; } - int permitsToRequest = Math.min(availablePermits, this.batchSize); + int permitsToRequest = Math.min(availablePermits, max); CurrentThroughputMode currentThroughputModeNow = this.currentThroughputMode; logger.trace("Trying to acquire partial batch of {} permits from {} available for {} in TM {}", permitsToRequest, availablePermits, this.id, currentThroughputModeNow); @@ -111,7 +111,7 @@ private int tryAcquirePartial() throws InterruptedException { return hasAcquiredPartial ? permitsToRequest : 0; } - private int requestInLowThroughputMode() throws InterruptedException { + private int requestInLowThroughputMode(int amount) throws InterruptedException { // Although LTM can be set / unset by many processes, only the MessageSource thread gets here, // so no actual concurrency logger.debug("Trying to acquire full permits for {}. Permits left: {}", this.id, @@ -120,11 +120,11 @@ private int requestInLowThroughputMode() throws InterruptedException { if (hasAcquired) { logger.debug("Acquired full permits for {}. Permits left: {}", this.id, this.semaphore.availablePermits()); // We've acquired all permits - there's no other process currently processing messages - if (!this.hasAcquiredFullPermits.compareAndSet(false, true)) { + if (this.lowThroughputPermitsAcquired.getAndSet(amount) != 0) { logger.warn("hasAcquiredFullPermits was already true. Permits left: {}", this.semaphore.availablePermits()); } - return this.batchSize; + return amount; } else { return 0; @@ -147,19 +147,22 @@ private boolean tryAcquire(int amount, CurrentThroughputMode currentThroughputMo } @Override - public void releaseBatch() { - maybeSwitchToLowThroughputMode(); - int permitsToRelease = getPermitsToRelease(this.batchSize); + public void release(int amount, ReleaseReason reason) { + logger.trace("Releasing {} permits ({}) for {}. Permits left: {}", amount, reason, this.id, + this.semaphore.availablePermits()); + switch (reason) { + case NONE_FETCHED -> maybeSwitchToLowThroughputMode(); + case PARTIAL_FETCH -> maybeSwitchToHighThroughputMode(amount); + case PROCESSED, LIMITED -> { + // No need to switch throughput mode + } + } + int permitsToRelease = getPermitsToRelease(amount); this.semaphore.release(permitsToRelease); - logger.trace("Released {} permits for {}. Permits left: {}", permitsToRelease, this.id, + logger.debug("Released {} permits ({}) for {}. Permits left: {}", permitsToRelease, reason, this.id, this.semaphore.availablePermits()); } - @Override - public int getBatchSize() { - return this.batchSize; - } - private void maybeSwitchToLowThroughputMode() { if (!BackPressureMode.FIXED_HIGH_THROUGHPUT.equals(this.backPressureConfiguration) && CurrentThroughputMode.HIGH.equals(this.currentThroughputMode)) { @@ -169,25 +172,6 @@ private void maybeSwitchToLowThroughputMode() { } } - @Override - public void release(int amount) { - logger.trace("Releasing {} permits for {}. Permits left: {}", amount, this.id, - this.semaphore.availablePermits()); - maybeSwitchToHighThroughputMode(amount); - int permitsToRelease = getPermitsToRelease(amount); - this.semaphore.release(permitsToRelease); - logger.trace("Released {} permits for {}. Permits left: {}", permitsToRelease, this.id, - this.semaphore.availablePermits()); - } - - private int getPermitsToRelease(int amount) { - return this.hasAcquiredFullPermits.compareAndSet(true, false) - // The first process that gets here should release all permits except for inflight messages - // We can have only one batch of messages at this point since we have all permits - ? this.totalPermits - (this.batchSize - amount) - : amount; - } - private void maybeSwitchToHighThroughputMode(int amount) { if (CurrentThroughputMode.LOW.equals(this.currentThroughputMode)) { logger.debug("{} unused permit(s), setting TM HIGH for {}. Permits left: {}", amount, this.id, @@ -196,6 +180,15 @@ private void maybeSwitchToHighThroughputMode(int amount) { } } + private int getPermitsToRelease(int amount) { + int lowThroughputPermits = this.lowThroughputPermitsAcquired.getAndSet(0); + return lowThroughputPermits > 0 + // The first process that gets here should release all permits except for inflight messages + // We can have only one batch of messages at this point since we have all permits + ? this.totalPermits - (lowThroughputPermits - amount) + : amount; + } + @Override public boolean drain(Duration timeout) { logger.debug("Waiting for up to {} seconds for approx. {} permits to be released for {}", timeout.getSeconds(), diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSource.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSource.java index e71dc4319..9041cd9c8 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSource.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSource.java @@ -17,6 +17,7 @@ import io.awspring.cloud.sqs.ConfigUtils; import io.awspring.cloud.sqs.listener.BackPressureHandler; +import io.awspring.cloud.sqs.listener.BackPressureHandler.ReleaseReason; import io.awspring.cloud.sqs.listener.BatchAwareBackPressureHandler; import io.awspring.cloud.sqs.listener.ContainerOptions; import io.awspring.cloud.sqs.listener.IdentifiableContainerComponent; @@ -214,7 +215,7 @@ private void pollAndEmitMessages() { if (!isRunning()) { logger.debug("MessageSource was stopped after permits where acquired. Returning {} permits", acquiredPermits); - this.backPressureHandler.release(acquiredPermits); + this.backPressureHandler.release(acquiredPermits, ReleaseReason.NONE_FETCHED); continue; } // @formatter:off @@ -252,15 +253,12 @@ private void handlePollBackOff() { protected abstract CompletableFuture> doPollForMessages(int messagesToRequest); public Collection> releaseUnusedPermits(int permits, Collection> msgs) { - if (msgs.isEmpty() && permits == this.backPressureHandler.getBatchSize()) { - this.backPressureHandler.releaseBatch(); - logger.trace("Released batch of unused permits for queue {}", this.pollingEndpointName); - } - else { - int permitsToRelease = permits - msgs.size(); - this.backPressureHandler.release(permitsToRelease); - logger.trace("Released {} unused permits for queue {}", permitsToRelease, this.pollingEndpointName); - } + int polledMessages = msgs.size(); + int permitsToRelease = permits - polledMessages; + ReleaseReason releaseReason = polledMessages == 0 ? ReleaseReason.NONE_FETCHED : ReleaseReason.PARTIAL_FETCH; + this.backPressureHandler.release(permitsToRelease, releaseReason); + logger.trace("Released {} unused ({}) permits for queue {} (messages polled {})", permitsToRelease, + releaseReason, this.pollingEndpointName, polledMessages); return msgs; } @@ -285,7 +283,7 @@ protected AcknowledgementCallback getAcknowledgementCallback() { private void releaseBackPressure() { logger.debug("Releasing permit for queue {}", this.pollingEndpointName); - this.backPressureHandler.release(1); + this.backPressureHandler.release(1, ReleaseReason.PROCESSED); } private Void handleSinkException(Throwable t) { From bd08a136f9e8e303d2d4f02ec71d66a27392869b Mon Sep 17 00:00:00 2001 From: Loic Rouchon Date: Thu, 6 Feb 2025 12:01:39 +0100 Subject: [PATCH 4/9] Remove BackPressureHandlerLimiter from the library and make it user-code in tests only (#1251) --- .../listener/AbstractContainerOptions.java | 19 +- ...tractPipelineMessageListenerContainer.java | 13 +- .../sqs/listener/BackPressureHandler.java | 12 + .../listener/BackPressureHandlerLimiter.java | 133 ----------- .../sqs/listener/BackPressureLimiter.java | 44 ---- .../BatchAwareBackPressureHandler.java | 24 ++ .../cloud/sqs/listener/ContainerOptions.java | 9 +- .../sqs/listener/ContainerOptionsBuilder.java | 8 +- .../sqs/integration/SqsIntegrationTests.java | 210 +++++++++++++++--- 9 files changed, 243 insertions(+), 229 deletions(-) delete mode 100644 spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java delete mode 100644 spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureLimiter.java diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractContainerOptions.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractContainerOptions.java index 2662b0ade..5beda1765 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractContainerOptions.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractContainerOptions.java @@ -20,6 +20,7 @@ import io.awspring.cloud.sqs.support.converter.MessagingMessageConverter; import io.awspring.cloud.sqs.support.converter.SqsMessagingMessageConverter; import java.time.Duration; +import java.util.function.Supplier; import org.springframework.core.task.TaskExecutor; import org.springframework.lang.Nullable; import org.springframework.retry.backoff.BackOffPolicy; @@ -55,7 +56,7 @@ public abstract class AbstractContainerOptions, private final BackPressureMode backPressureMode; - private final BackPressureLimiter backPressureLimiter; + private final Supplier backPressureHandlerSupplier; private final ListenerMode listenerMode; @@ -89,7 +90,7 @@ protected AbstractContainerOptions(Builder builder) { this.listenerShutdownTimeout = builder.listenerShutdownTimeout; this.acknowledgementShutdownTimeout = builder.acknowledgementShutdownTimeout; this.backPressureMode = builder.backPressureMode; - this.backPressureLimiter = builder.backPressureLimiter; + this.backPressureHandlerSupplier = builder.backPressureHandlerSupplier; this.listenerMode = builder.listenerMode; this.messageConverter = builder.messageConverter; this.acknowledgementMode = builder.acknowledgementMode; @@ -166,8 +167,8 @@ public BackPressureMode getBackPressureMode() { } @Override - public BackPressureLimiter getBackPressureLimiter() { - return this.backPressureLimiter; + public Supplier getBackPressureHandlerSupplier() { + return this.backPressureHandlerSupplier; } @Override @@ -232,7 +233,7 @@ protected abstract static class Builder, private static final BackPressureMode DEFAULT_THROUGHPUT_CONFIGURATION = BackPressureMode.AUTO; - private static final BackPressureLimiter DEFAULT_BACKPRESSURE_LIMITER = null; + private static final Supplier DEFAULT_BACKPRESSURE_LIMITER = null; private static final ListenerMode DEFAULT_MESSAGE_DELIVERY_STRATEGY = ListenerMode.SINGLE_MESSAGE; @@ -256,7 +257,7 @@ protected abstract static class Builder, private BackPressureMode backPressureMode = DEFAULT_THROUGHPUT_CONFIGURATION; - private BackPressureLimiter backPressureLimiter = DEFAULT_BACKPRESSURE_LIMITER; + private Supplier backPressureHandlerSupplier = DEFAULT_BACKPRESSURE_LIMITER; private Duration listenerShutdownTimeout = DEFAULT_LISTENER_SHUTDOWN_TIMEOUT; @@ -296,7 +297,7 @@ protected Builder(AbstractContainerOptions options) { this.listenerShutdownTimeout = options.listenerShutdownTimeout; this.acknowledgementShutdownTimeout = options.acknowledgementShutdownTimeout; this.backPressureMode = options.backPressureMode; - this.backPressureLimiter = options.backPressureLimiter; + this.backPressureHandlerSupplier = options.backPressureHandlerSupplier; this.listenerMode = options.listenerMode; this.messageConverter = options.messageConverter; this.acknowledgementMode = options.acknowledgementMode; @@ -397,8 +398,8 @@ public B backPressureMode(BackPressureMode backPressureMode) { } @Override - public B backPressureLimiter(BackPressureLimiter backPressureLimiter) { - this.backPressureLimiter = backPressureLimiter; + public B backPressureHandlerSupplier(Supplier backPressureHandlerSupplier) { + this.backPressureHandlerSupplier = backPressureHandlerSupplier; return self(); } diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java index 401f07248..5dcd5dd86 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java @@ -227,17 +227,14 @@ private TaskExecutor validateCustomExecutor(TaskExecutor taskExecutor) { protected BackPressureHandler createBackPressureHandler() { O containerOptions = getContainerOptions(); - List backPressureHandlers = new ArrayList<>(2); + if (containerOptions.getBackPressureHandlerSupplier() != null) { + return containerOptions.getBackPressureHandlerSupplier().get(); + } Duration acquireTimeout = containerOptions.getMaxDelayBetweenPolls(); int batchSize = containerOptions.getMaxMessagesPerPoll(); - backPressureHandlers.add(SemaphoreBackPressureHandler.builder().batchSize(batchSize) + return SemaphoreBackPressureHandler.builder().batchSize(batchSize) .totalPermits(containerOptions.getMaxConcurrentMessages()).acquireTimeout(acquireTimeout) - .throughputConfiguration(containerOptions.getBackPressureMode()).build()); - if (containerOptions.getBackPressureLimiter() != null) { - backPressureHandlers.add(new BackPressureHandlerLimiter(containerOptions.getBackPressureLimiter(), - acquireTimeout, containerOptions.getStandbyLimitPollingInterval(), batchSize)); - } - return new CompositeBackPressureHandler(backPressureHandlers, batchSize); + .throughputConfiguration(containerOptions.getBackPressureMode()).build(); } protected TaskExecutor createSourcesTaskExecutor() { diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java index f2ff274b1..a5921de68 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java @@ -57,6 +57,18 @@ public interface BackPressureHandler { */ void release(int amount, ReleaseReason reason); + /** + * Release the specified amount of permits. Each message that has been processed should release one permit, whether + * processing was successful or not. + * @param amount the amount of permits to release. + * + * @deprecated This method is deprecated and will not be called by the Spring Cloud AWS SQS listener anymore. + * Implement {@link #release(int, ReleaseReason)} instead. + */ + @Deprecated + default void release(int amount) { + } + /** * Attempts to acquire all permits up to the specified timeout. If successful, means all permits were returned and * thus no activity is left in the {@link io.awspring.cloud.sqs.listener.source.MessageSource}. diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java deleted file mode 100644 index cd031a129..000000000 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandlerLimiter.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Copyright 2013-2025 the original author or authors. - * - * Licensed 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 - * - * https://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 io.awspring.cloud.sqs.listener; - -import java.time.Duration; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * A {@link BatchAwareBackPressureHandler} implementation that uses an internal {@link Semaphore} for adapting the - * maximum number of permits that can be acquired by the {@link #backPressureHandler} based on the downstream - * backpressure limit computed by the {@link #backPressureLimiter}. - * - * @see BackPressureLimiter - */ -public class BackPressureHandlerLimiter implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { - - /** - * The {@link BackPressureLimiter} which computes a limit on how many permits can be requested at a given moment. - */ - private final BackPressureLimiter backPressureLimiter; - - /** - * The duration to wait for permits to be acquired. - */ - private final Duration acquireTimeout; - - /** - * The duration to sleep when the queue processing is in standby. - */ - private final Duration standbyLimitPollingInterval; - - /** - * The limit of permits that can be acquired at the current time. The permits limit is defined in the [0, - * Integer.MAX_VALUE] interval. A value of {@literal 0} means that no permits can be acquired. - *

- * This value is updated based on the downstream backpressure reported by the {@link #backPressureLimiter}. - */ - private final AtomicInteger permitsLimit = new AtomicInteger(0); - - private final ReducibleSemaphore semaphore = new ReducibleSemaphore(0); - - private final int batchSize; - - private String id; - - public BackPressureHandlerLimiter(BackPressureLimiter backPressureLimiter, Duration acquireTimeout, - Duration standbyLimitPollingInterval, int batchSize) { - this.backPressureLimiter = backPressureLimiter; - this.acquireTimeout = acquireTimeout; - this.standbyLimitPollingInterval = standbyLimitPollingInterval; - this.batchSize = batchSize; - } - - @Override - public void setId(String id) { - this.id = id; - } - - @Override - public String getId() { - return id; - } - - @Override - public int requestBatch() throws InterruptedException { - return request(batchSize); - } - - @Override - public int request(int amount) throws InterruptedException { - int permits = Math.min(updatePermitsLimit(), amount); - if (permits == 0) { - Thread.sleep(standbyLimitPollingInterval.toMillis()); - return 0; - } - if (semaphore.tryAcquire(permits, acquireTimeout.toMillis(), TimeUnit.MILLISECONDS)) { - return permits; - } - return 0; - } - - @Override - public void release(int amount, ReleaseReason reason) { - semaphore.release(amount); - } - - @Override - public boolean drain(Duration timeout) { - return true; - } - - private int updatePermitsLimit() { - return permitsLimit.updateAndGet(oldLimit -> { - int newLimit = Math.max(0, backPressureLimiter.limit()); - if (newLimit < oldLimit) { - int blockedPermits = oldLimit - newLimit; - semaphore.reducePermits(blockedPermits); - } - else if (newLimit > oldLimit) { - int releasedPermits = newLimit - oldLimit; - semaphore.release(releasedPermits); - } - return newLimit; - }); - } - - private static class ReducibleSemaphore extends Semaphore { - - ReducibleSemaphore(int permits) { - super(permits); - } - - @Override - public void reducePermits(int reduction) { - super.reducePermits(reduction); - } - } -} diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureLimiter.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureLimiter.java deleted file mode 100644 index f85ddba82..000000000 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureLimiter.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Copyright 2013-2024 the original author or authors. - * - * Licensed 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 - * - * https://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 io.awspring.cloud.sqs.listener; - -/** - * The BackPressureLimiter enables a dynamic reduction of the queues consumption capacity depending on external factors. - */ -public interface BackPressureLimiter { - - /** - * {@return the limit to be applied to the queue consumption.} - * - * The limit can be used to reduce the queue consumption capabilities of the next polling attempts. The container - * will work toward satisfying the limit by decreasing the maximum number of concurrent messages that can ve - * processed. - * - * The following values will have the following effects: - * - *

    - *
  • zero or negative limits will stop consumption from the queue. When such a situation occurs, the queue - * processing is said to be on "standby".
  • - *
  • Values >= 1 and < {@link ContainerOptions#getMaxConcurrentMessages()} will reduce the queue consumption - * capabilities of the next polling attempts.
  • - *
  • Values >= {@link ContainerOptions#getMaxConcurrentMessages()} will not reduce the queue consumption - * capabilities
  • - *
- * - * Note: the adjustment will require a few polling cycles to be in effect. - */ - int limit(); -} diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java index c9ce20f9b..06387976c 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java @@ -30,4 +30,28 @@ public interface BatchAwareBackPressureHandler extends BackPressureHandler { * @throws InterruptedException if the Thread is interrupted while waiting for permits. */ int requestBatch() throws InterruptedException; + + /** + * Release a batch of permits. This has the semantics of letting the {@link BackPressureHandler} know that all + * permits from a batch are being released, in opposition to {@link #release(int)} in which any number of permits + * can be specified. + * + * @deprecated This method is deprecated and will not be called by the Spring Cloud AWS SQS listener anymore. + * Implement {@link BackPressureHandler#release(int, ReleaseReason)} instead. + */ + @Deprecated + default void releaseBatch() { + } + + /** + * Return the configured batch size for this handler. + * @return the batch size. + * + * @deprecated This method is deprecated and will not be used by the Spring Cloud AWS SQS listener anymore. + */ + @Deprecated + default int getBatchSize() { + return 0; + } + } diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptions.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptions.java index e78f967a6..95921f33e 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptions.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptions.java @@ -20,6 +20,7 @@ import io.awspring.cloud.sqs.support.converter.MessagingMessageConverter; import java.time.Duration; import java.util.Collection; +import java.util.function.Supplier; import org.springframework.core.task.TaskExecutor; import org.springframework.lang.Nullable; import org.springframework.retry.backoff.BackOffPolicy; @@ -61,8 +62,6 @@ public interface ContainerOptions, B extends Co /** * {@return the amount of time to wait before checking again for the current limit when the queue processing is on * standby} Default is 100 milliseconds. - * - * @see BackPressureLimiter#limit() */ Duration getStandbyLimitPollingInterval(); @@ -136,10 +135,10 @@ default BackOffPolicy getPollBackOffPolicy() { BackPressureMode getBackPressureMode(); /** - * Return the {@link BackPressureLimiter} for this container. - * @return the backpressure limiter. + * Return the a {@link Supplier} to create a {@link BackPressureHandler} for this container. + * @return the BackPressureHandler supplier. */ - BackPressureLimiter getBackPressureLimiter(); + Supplier getBackPressureHandlerSupplier(); /** * Return the {@link ListenerMode} mode for this container. diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java index de88c0464..cf8c768d3 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java @@ -19,6 +19,7 @@ import io.awspring.cloud.sqs.listener.acknowledgement.handler.AcknowledgementMode; import io.awspring.cloud.sqs.support.converter.MessagingMessageConverter; import java.time.Duration; +import java.util.function.Supplier; import org.springframework.core.task.TaskExecutor; import org.springframework.retry.backoff.BackOffPolicy; @@ -62,7 +63,6 @@ public interface ContainerOptionsBuilder * * @param standbyLimitPollingInterval the limit polling interval when the queue processing is on standby. * @return this instance. - * @see BackPressureLimiter#limit() */ B standbyLimitPollingInterval(Duration standbyLimitPollingInterval); @@ -156,12 +156,12 @@ default B pollBackOffPolicy(BackOffPolicy pollBackOffPolicy) { B backPressureMode(BackPressureMode backPressureMode); /** - * Set the {@link BackPressureLimiter} for this container. Default is {@code null}. + * Set the {@link Supplier} of {@link BackPressureHandler} for this container. Default is {@code null}. * - * @param backPressureLimiter the backpressure limiter. + * @param backPressureHandlerSupplier the BackPressureHandler supplier. * @return this instance. */ - B backPressureLimiter(BackPressureLimiter backPressureLimiter); + B backPressureHandlerSupplier(Supplier backPressureHandlerSupplier); /** * Set the maximum interval between acknowledgements for batch acknowledgements. The default depends on the specific diff --git a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java index b9834b338..41a370511 100644 --- a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java +++ b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java @@ -27,11 +27,16 @@ import io.awspring.cloud.sqs.config.SqsBootstrapConfiguration; import io.awspring.cloud.sqs.config.SqsListenerConfigurer; import io.awspring.cloud.sqs.config.SqsMessageListenerContainerFactory; -import io.awspring.cloud.sqs.listener.BackPressureLimiter; +import io.awspring.cloud.sqs.listener.BackPressureHandler; +import io.awspring.cloud.sqs.listener.BackPressureMode; +import io.awspring.cloud.sqs.listener.BatchAwareBackPressureHandler; import io.awspring.cloud.sqs.listener.BatchVisibility; +import io.awspring.cloud.sqs.listener.CompositeBackPressureHandler; import io.awspring.cloud.sqs.listener.ContainerComponentFactory; +import io.awspring.cloud.sqs.listener.IdentifiableContainerComponent; import io.awspring.cloud.sqs.listener.MessageListenerContainer; import io.awspring.cloud.sqs.listener.QueueAttributes; +import io.awspring.cloud.sqs.listener.SemaphoreBackPressureHandler; import io.awspring.cloud.sqs.listener.SqsContainerOptions; import io.awspring.cloud.sqs.listener.SqsContainerOptionsBuilder; import io.awspring.cloud.sqs.listener.SqsHeaders; @@ -51,15 +56,22 @@ import io.awspring.cloud.sqs.listener.source.MessageSource; import io.awspring.cloud.sqs.operations.SqsTemplate; import java.lang.reflect.Method; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; import java.time.Duration; +import java.time.Instant; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Queue; import java.util.Random; import java.util.UUID; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.Semaphore; @@ -358,10 +370,12 @@ void maxConcurrentMessages() { assertDoesNotThrow(() -> latchContainer.maxConcurrentMessagesBarrier.await(10, TimeUnit.SECONDS)); } - static final class Limiter implements BackPressureLimiter { + static final class NonBlockingExternalConcurrencyLimiterBackPressureHandler implements BackPressureHandler { private final AtomicInteger limit; + private final AtomicInteger inFlight = new AtomicInteger(0); + private final AtomicBoolean draining = new AtomicBoolean(false); - Limiter(int max) { + NonBlockingExternalConcurrencyLimiterBackPressureHandler(int max) { limit = new AtomicInteger(max); } @@ -371,8 +385,34 @@ public void setLimit(int value) { } @Override - public int limit() { - return Math.max(0, limit.get()); + public int request(int amount) { + if (draining.get()) { + return 0; + } + int permits = Math.max(0, Math.min(limit.get() - inFlight.get(), amount)); + inFlight.addAndGet(permits); + return permits; + } + + @Override + public void release(int amount, ReleaseReason reason) { + inFlight.addAndGet(-amount); + } + + @Override + public boolean drain(Duration timeout) { + Duration drainingTimeout = Duration.ofSeconds(10L); + Duration drainingPollingIntervalCheck = Duration.ofMillis(50L); + draining.set(true); + limit.set(0); + Instant start = Instant.now(); + while (Duration.between(start, Instant.now()).compareTo(drainingTimeout) < 0) { + if (inFlight.get() == 0) { + return true; + } + sleep(drainingPollingIntervalCheck.toMillis()); + } + return false; } } @@ -382,7 +422,8 @@ void staticBackPressureLimitShouldCapQueueProcessingCapacity(int staticLimit, in throws Exception { AtomicInteger concurrentRequest = new AtomicInteger(); AtomicInteger maxConcurrentRequest = new AtomicInteger(); - Limiter limiter = new Limiter(staticLimit); + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( + staticLimit); String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_STATIC_LIMIT_" + staticLimit; IntStream.range(0, 10).forEach(index -> { List> messages = create10Messages("staticBackPressureLimit" + staticLimit); @@ -390,9 +431,17 @@ void staticBackPressureLimitShouldCapQueueProcessingCapacity(int staticLimit, in }); logger.debug("Sent 100 messages to queue {}", queueName); var latch = new CountDownLatch(100); - var container = SqsMessageListenerContainer.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()) - .queueNames(queueName).configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .maxConcurrentMessages(5).maxMessagesPerPoll(5).backPressureLimiter(limiter)) + var container = SqsMessageListenerContainer + .builder().sqsAsyncClient( + BaseSqsIntegrationTest.createAsyncClient()) + .queueNames( + queueName) + .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 5))) .messageListener(msg -> { int concurrentRqs = concurrentRequest.incrementAndGet(); maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); @@ -412,7 +461,8 @@ void staticBackPressureLimitShouldCapQueueProcessingCapacity(int staticLimit, in void zeroBackPressureLimitShouldStopQueueProcessing() throws Exception { AtomicInteger concurrentRequest = new AtomicInteger(); AtomicInteger maxConcurrentRequest = new AtomicInteger(); - Limiter limiter = new Limiter(0); + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( + 0); String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_STATIC_LIMIT_0"; IntStream.range(0, 10).forEach(index -> { List> messages = create10Messages("staticBackPressureLimit0"); @@ -420,9 +470,17 @@ void zeroBackPressureLimitShouldStopQueueProcessing() throws Exception { }); logger.debug("Sent 100 messages to queue {}", queueName); var latch = new CountDownLatch(100); - var container = SqsMessageListenerContainer.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()) - .queueNames(queueName).configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .maxConcurrentMessages(5).maxMessagesPerPoll(5).backPressureLimiter(limiter)) + var container = SqsMessageListenerContainer + .builder().sqsAsyncClient( + BaseSqsIntegrationTest.createAsyncClient()) + .queueNames( + queueName) + .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 5))) .messageListener(msg -> { int concurrentRqs = concurrentRequest.incrementAndGet(); maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); @@ -443,7 +501,8 @@ void zeroBackPressureLimitShouldStopQueueProcessing() throws Exception { void changeInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Exception { AtomicInteger concurrentRequest = new AtomicInteger(); AtomicInteger maxConcurrentRequest = new AtomicInteger(); - Limiter limiter = new Limiter(5); + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( + 5); String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_SYNC_ADAPTIVE_LIMIT"; int nbMessages = 280; IntStream.range(0, nbMessages / 10).forEach(index -> { @@ -454,9 +513,17 @@ void changeInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Except var latch = new CountDownLatch(nbMessages); var controlSemaphore = new Semaphore(0); var advanceSemaphore = new Semaphore(0); - var container = SqsMessageListenerContainer.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()) - .queueNames(queueName).configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .maxConcurrentMessages(5).maxMessagesPerPoll(5).backPressureLimiter(limiter)) + var container = SqsMessageListenerContainer + .builder().sqsAsyncClient( + BaseSqsIntegrationTest.createAsyncClient()) + .queueNames( + queueName) + .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 5))) .messageListener(msg -> { try { controlSemaphore.acquire(); @@ -476,10 +543,11 @@ void changeInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Except class Controller { private final Semaphore advanceSemaphore; private final Semaphore controlSemaphore; - private final Limiter limiter; + private final NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter; private final AtomicInteger maxConcurrentRequest; - Controller(Semaphore advanceSemaphore, Semaphore controlSemaphore, Limiter limiter, + Controller(Semaphore advanceSemaphore, Semaphore controlSemaphore, + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter, AtomicInteger maxConcurrentRequest) { this.advanceSemaphore = advanceSemaphore; this.controlSemaphore = controlSemaphore; @@ -559,6 +627,75 @@ void waitForAdvance(int permits) throws InterruptedException { } } + static class EventsCsvWriter { + private final Queue events = new ConcurrentLinkedQueue<>(List.of("event,time,value")); + + void registerEvent(String event, int value) { + events.add("%s,%s,%d".formatted(event, Instant.now(), value)); + } + + void write(Path path) throws Exception { + Files.writeString(path, String.join("\n", events), StandardCharsets.UTF_8, StandardOpenOption.CREATE, + StandardOpenOption.TRUNCATE_EXISTING); + } + } + + static class StatisticsBphDecorator implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { + private final BatchAwareBackPressureHandler delegate; + private final EventsCsvWriter eventCsv; + private String id; + + StatisticsBphDecorator(BatchAwareBackPressureHandler delegate, EventsCsvWriter eventsCsvWriter) { + this.delegate = delegate; + this.eventCsv = eventsCsvWriter; + } + + @Override + public int requestBatch() throws InterruptedException { + int permits = delegate.requestBatch(); + if (permits > 0) { + eventCsv.registerEvent("obtained_permits", permits); + } + return permits; + } + + @Override + public int request(int amount) throws InterruptedException { + int permits = delegate.request(amount); + if (permits > 0) { + eventCsv.registerEvent("obtained_permits", permits); + } + return permits; + } + + @Override + public void release(int amount, ReleaseReason reason) { + if (amount > 0) { + eventCsv.registerEvent("release_" + reason, amount); + } + delegate.release(amount, reason); + } + + @Override + public boolean drain(Duration timeout) { + eventCsv.registerEvent("drain", 1); + return delegate.drain(timeout); + } + + @Override + public void setId(String id) { + this.id = id; + if (delegate instanceof IdentifiableContainerComponent icc) { + icc.setId("delegate-" + id); + } + } + + @Override + public String getId() { + return id; + } + } + /** * This test simulates a progressive change in the back pressure limit. Unlike * {@link #changeInBackPressureLimitShouldAdaptQueueProcessingCapacity()}, this test does not block message @@ -578,7 +715,8 @@ void waitForAdvance(int permits) throws InterruptedException { void unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Exception { AtomicInteger concurrentRequest = new AtomicInteger(); AtomicInteger maxConcurrentRequest = new AtomicInteger(); - Limiter limiter = new Limiter(0); + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( + 0); String queueName = "REACTIVE_BACK_PRESSURE_LIMITER_QUEUE_NAME_ADAPTIVE_LIMIT"; int nbMessages = 1000; Semaphore advanceSemaphore = new Semaphore(0); @@ -588,11 +726,22 @@ void unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity( }); logger.debug("Sent {} messages to queue {}", nbMessages, queueName); var latch = new CountDownLatch(nbMessages); - var container = SqsMessageListenerContainer.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()) + EventsCsvWriter eventsCsvWriter = new EventsCsvWriter(); + var container = SqsMessageListenerContainer + .builder().sqsAsyncClient( + BaseSqsIntegrationTest.createAsyncClient()) .queueNames(queueName) - .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .standbyLimitPollingInterval(Duration.ofMillis(1)).maxConcurrentMessages(10) - .maxMessagesPerPoll(10).backPressureLimiter(limiter)) + .configure( + options -> options.pollTimeout(Duration.ofSeconds(1)) + .standbyLimitPollingInterval( + Duration.ofMillis(1)) + .backPressureHandlerSupplier(() -> new StatisticsBphDecorator( + new CompositeBackPressureHandler(List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(10).totalPermits(10) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 10), + eventsCsvWriter))) .messageListener(msg -> { int currentConcurrentRq = concurrentRequest.incrementAndGet(); maxConcurrentRequest.updateAndGet(max -> Math.max(max, currentConcurrentRq)); @@ -622,17 +771,26 @@ void unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity( while (latch.getCount() > 0 && changeLimitCount < nbMessages) { changeLimitCount++; int limit = progressiveLimitChange.applyAsInt(changeLimitCount); + int expectedMax = Math.min(10, limit); limiter.setLimit(limit); maxConcurrentRequest.set(0); - sleep(random.nextInt(10)); + sleep(random.nextInt(20)); int actualLimit = Math.min(10, limit); - int max = maxConcurrentRequest.getAndSet(0); + int max = maxConcurrentRequest.get(); if (max > 0) { // Ignore iterations where nothing was polled (messages consumption slower than iteration) limitsSum += actualLimit; maxConcurrentRqSum += max; } + eventsCsvWriter.registerEvent("max_concurrent_rq", max); + eventsCsvWriter.registerEvent("concurrent_rq", concurrentRequest.get()); + eventsCsvWriter.registerEvent("limit", limit); + eventsCsvWriter.registerEvent("in_flight", limiter.inFlight.get()); + eventsCsvWriter.registerEvent("expected_max", expectedMax); + eventsCsvWriter.registerEvent("max_minus_expected_max", max - expectedMax); } + eventsCsvWriter.write(Path.of( + "target/0-stats-unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity.csv")); assertThat(maxConcurrentRqSum).isLessThanOrEqualTo(limitsSum); assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); } From b8e041d8fba556bed554539ddcd9c3cd03884723 Mon Sep 17 00:00:00 2001 From: Loic Rouchon Date: Wed, 12 Feb 2025 14:48:46 +0100 Subject: [PATCH 5/9] Move SQS BackPressureHandlers tests to a dedicated integration test (#1251) --- .../SqsBackPressureIntegrationTests.java | 590 ++++++++++++++++++ .../sqs/integration/SqsIntegrationTests.java | 458 -------------- 2 files changed, 590 insertions(+), 458 deletions(-) create mode 100644 spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java diff --git a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java new file mode 100644 index 000000000..7fc18e308 --- /dev/null +++ b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java @@ -0,0 +1,590 @@ +/* + * Copyright 2013-2022 the original author or authors. + * + * Licensed 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 + * + * https://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 io.awspring.cloud.sqs.integration; + +import static java.util.Collections.singletonMap; +import static org.assertj.core.api.Assertions.assertThat; + +import io.awspring.cloud.sqs.config.SqsBootstrapConfiguration; +import io.awspring.cloud.sqs.listener.BackPressureHandler; +import io.awspring.cloud.sqs.listener.BackPressureMode; +import io.awspring.cloud.sqs.listener.BatchAwareBackPressureHandler; +import io.awspring.cloud.sqs.listener.CompositeBackPressureHandler; +import io.awspring.cloud.sqs.listener.IdentifiableContainerComponent; +import io.awspring.cloud.sqs.listener.SemaphoreBackPressureHandler; +import io.awspring.cloud.sqs.listener.SqsMessageListenerContainer; +import io.awspring.cloud.sqs.operations.SqsTemplate; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.time.Duration; +import java.time.Instant; +import java.util.List; +import java.util.Queue; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.IntUnaryOperator; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.boot.test.context.SpringBootTest; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.context.annotation.Import; +import org.springframework.messaging.Message; +import org.springframework.messaging.support.MessageBuilder; +import software.amazon.awssdk.services.sqs.SqsAsyncClient; +import software.amazon.awssdk.services.sqs.model.QueueAttributeName; + +/** + * Integration tests for SQS containers back pressure management. + * + * @author Loïc Rouchon + */ +@SpringBootTest +class SqsBackPressureIntegrationTests extends BaseSqsIntegrationTest { + + private static final Logger logger = LoggerFactory.getLogger(SqsBackPressureIntegrationTests.class); + + static final String RECEIVES_MESSAGE_QUEUE_NAME = "receives_message_test_queue"; + + static final String RECEIVES_MESSAGE_BATCH_QUEUE_NAME = "receives_message_batch_test_queue"; + + static final String RECEIVES_MESSAGE_ASYNC_QUEUE_NAME = "receives_message_async_test_queue"; + + static final String DOES_NOT_ACK_ON_ERROR_QUEUE_NAME = "does_not_ack_test_queue"; + + static final String DOES_NOT_ACK_ON_ERROR_ASYNC_QUEUE_NAME = "does_not_ack_async_test_queue"; + + static final String DOES_NOT_ACK_ON_ERROR_BATCH_QUEUE_NAME = "does_not_ack_batch_test_queue"; + + static final String DOES_NOT_ACK_ON_ERROR_BATCH_ASYNC_QUEUE_NAME = "does_not_ack_batch_async_test_queue"; + + static final String RESOLVES_PARAMETER_TYPES_QUEUE_NAME = "resolves_parameter_type_test_queue"; + + static final String MANUALLY_START_CONTAINER = "manually_start_container_test_queue"; + + static final String MANUALLY_CREATE_CONTAINER_QUEUE_NAME = "manually_create_container_test_queue"; + + static final String MANUALLY_CREATE_INACTIVE_CONTAINER_QUEUE_NAME = "manually_create_inactive_container_test_queue"; + + static final String MANUALLY_CREATE_FACTORY_QUEUE_NAME = "manually_create_factory_test_queue"; + + static final String CONSUMES_ONE_MESSAGE_AT_A_TIME_QUEUE_NAME = "consumes_one_message_test_queue"; + + static final String MAX_CONCURRENT_MESSAGES_QUEUE_NAME = "max_concurrent_messages_test_queue"; + + static final String LOW_RESOURCE_FACTORY = "lowResourceFactory"; + + static final String MANUAL_ACK_FACTORY = "manualAcknowledgementFactory"; + + static final String MANUAL_ACK_BATCH_FACTORY = "manualAcknowledgementBatchFactory"; + + static final String ACK_AFTER_SECOND_ERROR_FACTORY = "ackAfterSecondErrorFactory"; + + @BeforeAll + static void beforeTests() { + SqsAsyncClient client = createAsyncClient(); + CompletableFuture.allOf(createQueue(client, RECEIVES_MESSAGE_QUEUE_NAME), + createQueue(client, DOES_NOT_ACK_ON_ERROR_QUEUE_NAME, + singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "1")), + createQueue(client, DOES_NOT_ACK_ON_ERROR_ASYNC_QUEUE_NAME, + singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "1")), + createQueue(client, DOES_NOT_ACK_ON_ERROR_BATCH_QUEUE_NAME, + singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "1")), + createQueue(client, DOES_NOT_ACK_ON_ERROR_BATCH_ASYNC_QUEUE_NAME, + singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "1")), + createQueue(client, RECEIVES_MESSAGE_ASYNC_QUEUE_NAME), + createQueue(client, RECEIVES_MESSAGE_BATCH_QUEUE_NAME), + createQueue(client, RESOLVES_PARAMETER_TYPES_QUEUE_NAME, + singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "20")), + createQueue(client, MANUALLY_CREATE_CONTAINER_QUEUE_NAME), + createQueue(client, MANUALLY_CREATE_INACTIVE_CONTAINER_QUEUE_NAME), + createQueue(client, MANUALLY_CREATE_FACTORY_QUEUE_NAME), + createQueue(client, CONSUMES_ONE_MESSAGE_AT_A_TIME_QUEUE_NAME), + createQueue(client, MAX_CONCURRENT_MESSAGES_QUEUE_NAME)).join(); + } + + @Autowired + SqsTemplate sqsTemplate; + + static final class NonBlockingExternalConcurrencyLimiterBackPressureHandler implements BackPressureHandler { + private final AtomicInteger limit; + private final AtomicInteger inFlight = new AtomicInteger(0); + private final AtomicBoolean draining = new AtomicBoolean(false); + + NonBlockingExternalConcurrencyLimiterBackPressureHandler(int max) { + limit = new AtomicInteger(max); + } + + public void setLimit(int value) { + logger.info("adjusting limit from {} to {}", limit.get(), value); + limit.set(value); + } + + @Override + public int request(int amount) { + if (draining.get()) { + return 0; + } + int permits = Math.max(0, Math.min(limit.get() - inFlight.get(), amount)); + inFlight.addAndGet(permits); + return permits; + } + + @Override + public void release(int amount, ReleaseReason reason) { + inFlight.addAndGet(-amount); + } + + @Override + public boolean drain(Duration timeout) { + Duration drainingTimeout = Duration.ofSeconds(10L); + Duration drainingPollingIntervalCheck = Duration.ofMillis(50L); + draining.set(true); + limit.set(0); + Instant start = Instant.now(); + while (Duration.between(start, Instant.now()).compareTo(drainingTimeout) < 0) { + if (inFlight.get() == 0) { + return true; + } + sleep(drainingPollingIntervalCheck.toMillis()); + } + return false; + } + } + + @ParameterizedTest + @CsvSource({ "2,2", "4,4", "5,5", "20,5" }) + void staticBackPressureLimitShouldCapQueueProcessingCapacity(int staticLimit, int expectedMaxConcurrentRequests) + throws Exception { + AtomicInteger concurrentRequest = new AtomicInteger(); + AtomicInteger maxConcurrentRequest = new AtomicInteger(); + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( + staticLimit); + String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_STATIC_LIMIT_" + staticLimit; + IntStream.range(0, 10).forEach(index -> { + List> messages = create10Messages("staticBackPressureLimit" + staticLimit); + sqsTemplate.sendMany(queueName, messages); + }); + logger.debug("Sent 100 messages to queue {}", queueName); + var latch = new CountDownLatch(100); + var container = SqsMessageListenerContainer + .builder().sqsAsyncClient( + BaseSqsIntegrationTest.createAsyncClient()) + .queueNames( + queueName) + .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 5))) + .messageListener(msg -> { + int concurrentRqs = concurrentRequest.incrementAndGet(); + maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); + sleep(50L); + logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), + maxConcurrentRequest.get(), latch.getCount()); + latch.countDown(); + concurrentRequest.decrementAndGet(); + }).build(); + container.start(); + assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); + assertThat(maxConcurrentRequest.get()).isEqualTo(expectedMaxConcurrentRequests); + container.stop(); + } + + @Test + void zeroBackPressureLimitShouldStopQueueProcessing() throws Exception { + AtomicInteger concurrentRequest = new AtomicInteger(); + AtomicInteger maxConcurrentRequest = new AtomicInteger(); + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( + 0); + String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_STATIC_LIMIT_0"; + IntStream.range(0, 10).forEach(index -> { + List> messages = create10Messages("staticBackPressureLimit0"); + sqsTemplate.sendMany(queueName, messages); + }); + logger.debug("Sent 100 messages to queue {}", queueName); + var latch = new CountDownLatch(100); + var container = SqsMessageListenerContainer + .builder().sqsAsyncClient( + BaseSqsIntegrationTest.createAsyncClient()) + .queueNames( + queueName) + .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 5))) + .messageListener(msg -> { + int concurrentRqs = concurrentRequest.incrementAndGet(); + maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); + sleep(50L); + logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), + maxConcurrentRequest.get(), latch.getCount()); + latch.countDown(); + concurrentRequest.decrementAndGet(); + }).build(); + container.start(); + assertThat(latch.await(2, TimeUnit.SECONDS)).isFalse(); + assertThat(maxConcurrentRequest.get()).isZero(); + assertThat(latch.getCount()).isEqualTo(100L); + container.stop(); + } + + @Test + void changeInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Exception { + AtomicInteger concurrentRequest = new AtomicInteger(); + AtomicInteger maxConcurrentRequest = new AtomicInteger(); + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( + 5); + String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_SYNC_ADAPTIVE_LIMIT"; + int nbMessages = 280; + IntStream.range(0, nbMessages / 10).forEach(index -> { + List> messages = create10Messages("syncAdaptiveBackPressureLimit"); + sqsTemplate.sendMany(queueName, messages); + }); + logger.debug("Sent {} messages to queue {}", nbMessages, queueName); + var latch = new CountDownLatch(nbMessages); + var controlSemaphore = new Semaphore(0); + var advanceSemaphore = new Semaphore(0); + var container = SqsMessageListenerContainer + .builder().sqsAsyncClient( + BaseSqsIntegrationTest.createAsyncClient()) + .queueNames( + queueName) + .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 5))) + .messageListener(msg -> { + try { + controlSemaphore.acquire(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + int concurrentRqs = concurrentRequest.incrementAndGet(); + maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); + latch.countDown(); + logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), + maxConcurrentRequest.get(), latch.getCount()); + sleep(10L); + concurrentRequest.decrementAndGet(); + advanceSemaphore.release(); + }).build(); + class Controller { + private final Semaphore advanceSemaphore; + private final Semaphore controlSemaphore; + private final NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter; + private final AtomicInteger maxConcurrentRequest; + + Controller(Semaphore advanceSemaphore, Semaphore controlSemaphore, + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter, + AtomicInteger maxConcurrentRequest) { + this.advanceSemaphore = advanceSemaphore; + this.controlSemaphore = controlSemaphore; + this.limiter = limiter; + this.maxConcurrentRequest = maxConcurrentRequest; + } + + public void updateLimit(int newLimit) { + limiter.setLimit(newLimit); + } + + void updateLimitAndWaitForReset(int newLimit) throws InterruptedException { + updateLimit(newLimit); + int atLeastTwoPollingCycles = 2 * 5; + controlSemaphore.release(atLeastTwoPollingCycles); + waitForAdvance(atLeastTwoPollingCycles); + maxConcurrentRequest.set(0); + } + + void advance(int permits) { + controlSemaphore.release(permits); + } + + void waitForAdvance(int permits) throws InterruptedException { + assertThat(advanceSemaphore.tryAcquire(permits, 5, TimeUnit.SECONDS)) + .withFailMessage(() -> "Waiting for %d permits timed out. Only %d permits available" + .formatted(permits, advanceSemaphore.availablePermits())) + .isTrue(); + } + } + var controller = new Controller(advanceSemaphore, controlSemaphore, limiter, maxConcurrentRequest); + try { + container.start(); + + controller.advance(50); + controller.waitForAdvance(50); + // not limiting queue processing capacity + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); + controller.updateLimitAndWaitForReset(2); + controller.advance(50); + + controller.waitForAdvance(50); + // limiting queue processing capacity + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(2); + controller.updateLimitAndWaitForReset(7); + controller.advance(50); + + controller.waitForAdvance(50); + // not limiting queue processing capacity + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); + controller.updateLimitAndWaitForReset(3); + controller.advance(50); + sleep(10L); + limiter.setLimit(1); + sleep(10L); + limiter.setLimit(2); + sleep(10L); + limiter.setLimit(3); + + controller.waitForAdvance(50); + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(3); + // stopping processing of the queue + controller.updateLimit(0); + controller.advance(50); + assertThat(advanceSemaphore.tryAcquire(10, 5, TimeUnit.SECONDS)) + .withFailMessage("Acquiring semaphore should have timed out as limit was set to 0").isFalse(); + + // resume queue processing + controller.updateLimit(6); + + controller.waitForAdvance(50); + assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); + assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); + } + finally { + container.stop(); + } + } + + static class EventsCsvWriter { + private final Queue events = new ConcurrentLinkedQueue<>(List.of("event,time,value")); + + void registerEvent(String event, int value) { + events.add("%s,%s,%d".formatted(event, Instant.now(), value)); + } + + void write(Path path) throws Exception { + Files.writeString(path, String.join("\n", events), StandardCharsets.UTF_8, StandardOpenOption.CREATE, + StandardOpenOption.TRUNCATE_EXISTING); + } + } + + static class StatisticsBphDecorator implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { + private final BatchAwareBackPressureHandler delegate; + private final EventsCsvWriter eventCsv; + private String id; + + StatisticsBphDecorator(BatchAwareBackPressureHandler delegate, EventsCsvWriter eventsCsvWriter) { + this.delegate = delegate; + this.eventCsv = eventsCsvWriter; + } + + @Override + public int requestBatch() throws InterruptedException { + int permits = delegate.requestBatch(); + if (permits > 0) { + eventCsv.registerEvent("obtained_permits", permits); + } + return permits; + } + + @Override + public int request(int amount) throws InterruptedException { + int permits = delegate.request(amount); + if (permits > 0) { + eventCsv.registerEvent("obtained_permits", permits); + } + return permits; + } + + @Override + public void release(int amount, ReleaseReason reason) { + if (amount > 0) { + eventCsv.registerEvent("release_" + reason, amount); + } + delegate.release(amount, reason); + } + + @Override + public boolean drain(Duration timeout) { + eventCsv.registerEvent("drain", 1); + return delegate.drain(timeout); + } + + @Override + public void setId(String id) { + this.id = id; + if (delegate instanceof IdentifiableContainerComponent icc) { + icc.setId("delegate-" + id); + } + } + + @Override + public String getId() { + return id; + } + } + + /** + * This test simulates a progressive change in the back pressure limit. Unlike + * {@link #changeInBackPressureLimitShouldAdaptQueueProcessingCapacity()}, this test does not block message + * consumption while updating the limit. + *

+ * The limit is updated in a loop until all messages are consumed. The update follows a triangle wave pattern with a + * minimum of 0, a maximum of 15, and a period of 30 iterations. After each update of the limit, the test waits up + * to 10ms and samples the maximum number of concurrent messages that were processed since the update. This number + * can be higher than the defined limit during the adaptation period of the decreasing limit wave. For the + * increasing limit wave, it is usually lower due to the adaptation delay. In both cases, the maximum number of + * concurrent messages being processed rapidly converges toward the defined limit. + *

+ * The test passes if the sum of the sampled maximum number of concurrently processed messages is lower than the sum + * of the limits at those points in time. + */ + @Test + void unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Exception { + AtomicInteger concurrentRequest = new AtomicInteger(); + AtomicInteger maxConcurrentRequest = new AtomicInteger(); + NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( + 0); + String queueName = "REACTIVE_BACK_PRESSURE_LIMITER_QUEUE_NAME_ADAPTIVE_LIMIT"; + int nbMessages = 1000; + Semaphore advanceSemaphore = new Semaphore(0); + IntStream.range(0, nbMessages / 10).forEach(index -> { + List> messages = create10Messages("reactAdaptiveBackPressureLimit"); + sqsTemplate.sendMany(queueName, messages); + }); + logger.debug("Sent {} messages to queue {}", nbMessages, queueName); + var latch = new CountDownLatch(nbMessages); + EventsCsvWriter eventsCsvWriter = new EventsCsvWriter(); + var container = SqsMessageListenerContainer + .builder().sqsAsyncClient( + BaseSqsIntegrationTest.createAsyncClient()) + .queueNames(queueName) + .configure( + options -> options.pollTimeout(Duration.ofSeconds(1)) + .standbyLimitPollingInterval( + Duration.ofMillis(1)) + .backPressureHandlerSupplier(() -> new StatisticsBphDecorator( + new CompositeBackPressureHandler(List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(10).totalPermits(10) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 10), + eventsCsvWriter))) + .messageListener(msg -> { + int currentConcurrentRq = concurrentRequest.incrementAndGet(); + maxConcurrentRequest.updateAndGet(max -> Math.max(max, currentConcurrentRq)); + sleep(ThreadLocalRandom.current().nextInt(10)); + latch.countDown(); + logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), + maxConcurrentRequest.get(), latch.getCount()); + concurrentRequest.decrementAndGet(); + advanceSemaphore.release(); + }).build(); + IntUnaryOperator progressiveLimitChange = (int x) -> { + int period = 30; + int halfPeriod = period / 2; + if (x % period < halfPeriod) { + return (x % halfPeriod); + } + else { + return (halfPeriod - (x % halfPeriod)); + } + }; + try { + container.start(); + Random random = new Random(); + int limitsSum = 0; + int maxConcurrentRqSum = 0; + int changeLimitCount = 0; + while (latch.getCount() > 0 && changeLimitCount < nbMessages) { + changeLimitCount++; + int limit = progressiveLimitChange.applyAsInt(changeLimitCount); + int expectedMax = Math.min(10, limit); + limiter.setLimit(limit); + maxConcurrentRequest.set(0); + sleep(random.nextInt(20)); + int actualLimit = Math.min(10, limit); + int max = maxConcurrentRequest.get(); + if (max > 0) { + // Ignore iterations where nothing was polled (messages consumption slower than iteration) + limitsSum += actualLimit; + maxConcurrentRqSum += max; + } + eventsCsvWriter.registerEvent("max_concurrent_rq", max); + eventsCsvWriter.registerEvent("concurrent_rq", concurrentRequest.get()); + eventsCsvWriter.registerEvent("limit", limit); + eventsCsvWriter.registerEvent("in_flight", limiter.inFlight.get()); + eventsCsvWriter.registerEvent("expected_max", expectedMax); + eventsCsvWriter.registerEvent("max_minus_expected_max", max - expectedMax); + } + eventsCsvWriter.write(Path.of( + "target/0-stats-unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity.csv")); + assertThat(maxConcurrentRqSum).isLessThanOrEqualTo(limitsSum); + assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); + } + finally { + container.stop(); + } + } + + private static void sleep(long millis) { + try { + Thread.sleep(millis); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + private List> create10Messages(String testName) { + return IntStream.range(0, 10).mapToObj(index -> testName + "-payload-" + index) + .map(payload -> MessageBuilder.withPayload(payload).build()).collect(Collectors.toList()); + } + + @Import(SqsBootstrapConfiguration.class) + @Configuration + static class SQSConfiguration { + + @Bean + SqsTemplate sqsTemplate() { + return SqsTemplate.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()).build(); + } + } +} diff --git a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java index 41a370511..76a7a65f7 100644 --- a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java +++ b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsIntegrationTests.java @@ -27,16 +27,10 @@ import io.awspring.cloud.sqs.config.SqsBootstrapConfiguration; import io.awspring.cloud.sqs.config.SqsListenerConfigurer; import io.awspring.cloud.sqs.config.SqsMessageListenerContainerFactory; -import io.awspring.cloud.sqs.listener.BackPressureHandler; -import io.awspring.cloud.sqs.listener.BackPressureMode; -import io.awspring.cloud.sqs.listener.BatchAwareBackPressureHandler; import io.awspring.cloud.sqs.listener.BatchVisibility; -import io.awspring.cloud.sqs.listener.CompositeBackPressureHandler; import io.awspring.cloud.sqs.listener.ContainerComponentFactory; -import io.awspring.cloud.sqs.listener.IdentifiableContainerComponent; import io.awspring.cloud.sqs.listener.MessageListenerContainer; import io.awspring.cloud.sqs.listener.QueueAttributes; -import io.awspring.cloud.sqs.listener.SemaphoreBackPressureHandler; import io.awspring.cloud.sqs.listener.SqsContainerOptions; import io.awspring.cloud.sqs.listener.SqsContainerOptionsBuilder; import io.awspring.cloud.sqs.listener.SqsHeaders; @@ -56,36 +50,22 @@ import io.awspring.cloud.sqs.listener.source.MessageSource; import io.awspring.cloud.sqs.operations.SqsTemplate; import java.lang.reflect.Method; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.StandardOpenOption; import java.time.Duration; -import java.time.Instant; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.Queue; -import java.util.Random; import java.util.UUID; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.Semaphore; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.IntUnaryOperator; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -370,444 +350,6 @@ void maxConcurrentMessages() { assertDoesNotThrow(() -> latchContainer.maxConcurrentMessagesBarrier.await(10, TimeUnit.SECONDS)); } - static final class NonBlockingExternalConcurrencyLimiterBackPressureHandler implements BackPressureHandler { - private final AtomicInteger limit; - private final AtomicInteger inFlight = new AtomicInteger(0); - private final AtomicBoolean draining = new AtomicBoolean(false); - - NonBlockingExternalConcurrencyLimiterBackPressureHandler(int max) { - limit = new AtomicInteger(max); - } - - public void setLimit(int value) { - logger.info("adjusting limit from {} to {}", limit.get(), value); - limit.set(value); - } - - @Override - public int request(int amount) { - if (draining.get()) { - return 0; - } - int permits = Math.max(0, Math.min(limit.get() - inFlight.get(), amount)); - inFlight.addAndGet(permits); - return permits; - } - - @Override - public void release(int amount, ReleaseReason reason) { - inFlight.addAndGet(-amount); - } - - @Override - public boolean drain(Duration timeout) { - Duration drainingTimeout = Duration.ofSeconds(10L); - Duration drainingPollingIntervalCheck = Duration.ofMillis(50L); - draining.set(true); - limit.set(0); - Instant start = Instant.now(); - while (Duration.between(start, Instant.now()).compareTo(drainingTimeout) < 0) { - if (inFlight.get() == 0) { - return true; - } - sleep(drainingPollingIntervalCheck.toMillis()); - } - return false; - } - } - - @ParameterizedTest - @CsvSource({ "2,2", "4,4", "5,5", "20,5" }) - void staticBackPressureLimitShouldCapQueueProcessingCapacity(int staticLimit, int expectedMaxConcurrentRequests) - throws Exception { - AtomicInteger concurrentRequest = new AtomicInteger(); - AtomicInteger maxConcurrentRequest = new AtomicInteger(); - NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( - staticLimit); - String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_STATIC_LIMIT_" + staticLimit; - IntStream.range(0, 10).forEach(index -> { - List> messages = create10Messages("staticBackPressureLimit" + staticLimit); - sqsTemplate.sendMany(queueName, messages); - }); - logger.debug("Sent 100 messages to queue {}", queueName); - var latch = new CountDownLatch(100); - var container = SqsMessageListenerContainer - .builder().sqsAsyncClient( - BaseSqsIntegrationTest.createAsyncClient()) - .queueNames( - queueName) - .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) - .acquireTimeout(Duration.ofSeconds(1L)) - .throughputConfiguration(BackPressureMode.AUTO).build()), - 5))) - .messageListener(msg -> { - int concurrentRqs = concurrentRequest.incrementAndGet(); - maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); - sleep(50L); - logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), - maxConcurrentRequest.get(), latch.getCount()); - latch.countDown(); - concurrentRequest.decrementAndGet(); - }).build(); - container.start(); - assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); - assertThat(maxConcurrentRequest.get()).isEqualTo(expectedMaxConcurrentRequests); - container.stop(); - } - - @Test - void zeroBackPressureLimitShouldStopQueueProcessing() throws Exception { - AtomicInteger concurrentRequest = new AtomicInteger(); - AtomicInteger maxConcurrentRequest = new AtomicInteger(); - NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( - 0); - String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_STATIC_LIMIT_0"; - IntStream.range(0, 10).forEach(index -> { - List> messages = create10Messages("staticBackPressureLimit0"); - sqsTemplate.sendMany(queueName, messages); - }); - logger.debug("Sent 100 messages to queue {}", queueName); - var latch = new CountDownLatch(100); - var container = SqsMessageListenerContainer - .builder().sqsAsyncClient( - BaseSqsIntegrationTest.createAsyncClient()) - .queueNames( - queueName) - .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) - .acquireTimeout(Duration.ofSeconds(1L)) - .throughputConfiguration(BackPressureMode.AUTO).build()), - 5))) - .messageListener(msg -> { - int concurrentRqs = concurrentRequest.incrementAndGet(); - maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); - sleep(50L); - logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), - maxConcurrentRequest.get(), latch.getCount()); - latch.countDown(); - concurrentRequest.decrementAndGet(); - }).build(); - container.start(); - assertThat(latch.await(2, TimeUnit.SECONDS)).isFalse(); - assertThat(maxConcurrentRequest.get()).isZero(); - assertThat(latch.getCount()).isEqualTo(100L); - container.stop(); - } - - @Test - void changeInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Exception { - AtomicInteger concurrentRequest = new AtomicInteger(); - AtomicInteger maxConcurrentRequest = new AtomicInteger(); - NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( - 5); - String queueName = "BACK_PRESSURE_LIMITER_QUEUE_NAME_SYNC_ADAPTIVE_LIMIT"; - int nbMessages = 280; - IntStream.range(0, nbMessages / 10).forEach(index -> { - List> messages = create10Messages("syncAdaptiveBackPressureLimit"); - sqsTemplate.sendMany(queueName, messages); - }); - logger.debug("Sent {} messages to queue {}", nbMessages, queueName); - var latch = new CountDownLatch(nbMessages); - var controlSemaphore = new Semaphore(0); - var advanceSemaphore = new Semaphore(0); - var container = SqsMessageListenerContainer - .builder().sqsAsyncClient( - BaseSqsIntegrationTest.createAsyncClient()) - .queueNames( - queueName) - .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) - .acquireTimeout(Duration.ofSeconds(1L)) - .throughputConfiguration(BackPressureMode.AUTO).build()), - 5))) - .messageListener(msg -> { - try { - controlSemaphore.acquire(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - int concurrentRqs = concurrentRequest.incrementAndGet(); - maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); - latch.countDown(); - logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), - maxConcurrentRequest.get(), latch.getCount()); - sleep(10L); - concurrentRequest.decrementAndGet(); - advanceSemaphore.release(); - }).build(); - class Controller { - private final Semaphore advanceSemaphore; - private final Semaphore controlSemaphore; - private final NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter; - private final AtomicInteger maxConcurrentRequest; - - Controller(Semaphore advanceSemaphore, Semaphore controlSemaphore, - NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter, - AtomicInteger maxConcurrentRequest) { - this.advanceSemaphore = advanceSemaphore; - this.controlSemaphore = controlSemaphore; - this.limiter = limiter; - this.maxConcurrentRequest = maxConcurrentRequest; - } - - public void updateLimit(int newLimit) { - limiter.setLimit(newLimit); - } - - void updateLimitAndWaitForReset(int newLimit) throws InterruptedException { - updateLimit(newLimit); - int atLeastTwoPollingCycles = 2 * 5; - controlSemaphore.release(atLeastTwoPollingCycles); - waitForAdvance(atLeastTwoPollingCycles); - maxConcurrentRequest.set(0); - } - - void advance(int permits) { - controlSemaphore.release(permits); - } - - void waitForAdvance(int permits) throws InterruptedException { - assertThat(advanceSemaphore.tryAcquire(permits, 5, TimeUnit.SECONDS)) - .withFailMessage(() -> "Waiting for %d permits timed out. Only %d permits available" - .formatted(permits, advanceSemaphore.availablePermits())) - .isTrue(); - } - } - var controller = new Controller(advanceSemaphore, controlSemaphore, limiter, maxConcurrentRequest); - try { - container.start(); - - controller.advance(50); - controller.waitForAdvance(50); - // not limiting queue processing capacity - assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); - controller.updateLimitAndWaitForReset(2); - controller.advance(50); - - controller.waitForAdvance(50); - // limiting queue processing capacity - assertThat(controller.maxConcurrentRequest.get()).isEqualTo(2); - controller.updateLimitAndWaitForReset(7); - controller.advance(50); - - controller.waitForAdvance(50); - // not limiting queue processing capacity - assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); - controller.updateLimitAndWaitForReset(3); - controller.advance(50); - sleep(10L); - limiter.setLimit(1); - sleep(10L); - limiter.setLimit(2); - sleep(10L); - limiter.setLimit(3); - - controller.waitForAdvance(50); - assertThat(controller.maxConcurrentRequest.get()).isEqualTo(3); - // stopping processing of the queue - controller.updateLimit(0); - controller.advance(50); - assertThat(advanceSemaphore.tryAcquire(10, 5, TimeUnit.SECONDS)) - .withFailMessage("Acquiring semaphore should have timed out as limit was set to 0").isFalse(); - - // resume queue processing - controller.updateLimit(6); - - controller.waitForAdvance(50); - assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); - assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); - } - finally { - container.stop(); - } - } - - static class EventsCsvWriter { - private final Queue events = new ConcurrentLinkedQueue<>(List.of("event,time,value")); - - void registerEvent(String event, int value) { - events.add("%s,%s,%d".formatted(event, Instant.now(), value)); - } - - void write(Path path) throws Exception { - Files.writeString(path, String.join("\n", events), StandardCharsets.UTF_8, StandardOpenOption.CREATE, - StandardOpenOption.TRUNCATE_EXISTING); - } - } - - static class StatisticsBphDecorator implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { - private final BatchAwareBackPressureHandler delegate; - private final EventsCsvWriter eventCsv; - private String id; - - StatisticsBphDecorator(BatchAwareBackPressureHandler delegate, EventsCsvWriter eventsCsvWriter) { - this.delegate = delegate; - this.eventCsv = eventsCsvWriter; - } - - @Override - public int requestBatch() throws InterruptedException { - int permits = delegate.requestBatch(); - if (permits > 0) { - eventCsv.registerEvent("obtained_permits", permits); - } - return permits; - } - - @Override - public int request(int amount) throws InterruptedException { - int permits = delegate.request(amount); - if (permits > 0) { - eventCsv.registerEvent("obtained_permits", permits); - } - return permits; - } - - @Override - public void release(int amount, ReleaseReason reason) { - if (amount > 0) { - eventCsv.registerEvent("release_" + reason, amount); - } - delegate.release(amount, reason); - } - - @Override - public boolean drain(Duration timeout) { - eventCsv.registerEvent("drain", 1); - return delegate.drain(timeout); - } - - @Override - public void setId(String id) { - this.id = id; - if (delegate instanceof IdentifiableContainerComponent icc) { - icc.setId("delegate-" + id); - } - } - - @Override - public String getId() { - return id; - } - } - - /** - * This test simulates a progressive change in the back pressure limit. Unlike - * {@link #changeInBackPressureLimitShouldAdaptQueueProcessingCapacity()}, this test does not block message - * consumption while updating the limit. - *

- * The limit is updated in a loop until all messages are consumed. The update follows a triangle wave pattern with a - * minimum of 0, a maximum of 15, and a period of 30 iterations. After each update of the limit, the test waits up - * to 10ms and samples the maximum number of concurrent messages that were processed since the update. This number - * can be higher than the defined limit during the adaptation period of the decreasing limit wave. For the - * increasing limit wave, it is usually lower due to the adaptation delay. In both cases, the maximum number of - * concurrent messages being processed rapidly converges toward the defined limit. - *

- * The test passes if the sum of the sampled maximum number of concurrently processed messages is lower than the sum - * of the limits at those points in time. - */ - @Test - void unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Exception { - AtomicInteger concurrentRequest = new AtomicInteger(); - AtomicInteger maxConcurrentRequest = new AtomicInteger(); - NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter = new NonBlockingExternalConcurrencyLimiterBackPressureHandler( - 0); - String queueName = "REACTIVE_BACK_PRESSURE_LIMITER_QUEUE_NAME_ADAPTIVE_LIMIT"; - int nbMessages = 1000; - Semaphore advanceSemaphore = new Semaphore(0); - IntStream.range(0, nbMessages / 10).forEach(index -> { - List> messages = create10Messages("reactAdaptiveBackPressureLimit"); - sqsTemplate.sendMany(queueName, messages); - }); - logger.debug("Sent {} messages to queue {}", nbMessages, queueName); - var latch = new CountDownLatch(nbMessages); - EventsCsvWriter eventsCsvWriter = new EventsCsvWriter(); - var container = SqsMessageListenerContainer - .builder().sqsAsyncClient( - BaseSqsIntegrationTest.createAsyncClient()) - .queueNames(queueName) - .configure( - options -> options.pollTimeout(Duration.ofSeconds(1)) - .standbyLimitPollingInterval( - Duration.ofMillis(1)) - .backPressureHandlerSupplier(() -> new StatisticsBphDecorator( - new CompositeBackPressureHandler(List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(10).totalPermits(10) - .acquireTimeout(Duration.ofSeconds(1L)) - .throughputConfiguration(BackPressureMode.AUTO).build()), - 10), - eventsCsvWriter))) - .messageListener(msg -> { - int currentConcurrentRq = concurrentRequest.incrementAndGet(); - maxConcurrentRequest.updateAndGet(max -> Math.max(max, currentConcurrentRq)); - sleep(ThreadLocalRandom.current().nextInt(10)); - latch.countDown(); - logger.debug("concurrent rq {}, max concurrent rq {}, latch count {}", concurrentRequest.get(), - maxConcurrentRequest.get(), latch.getCount()); - concurrentRequest.decrementAndGet(); - advanceSemaphore.release(); - }).build(); - IntUnaryOperator progressiveLimitChange = (int x) -> { - int period = 30; - int halfPeriod = period / 2; - if (x % period < halfPeriod) { - return (x % halfPeriod); - } - else { - return (halfPeriod - (x % halfPeriod)); - } - }; - try { - container.start(); - Random random = new Random(); - int limitsSum = 0; - int maxConcurrentRqSum = 0; - int changeLimitCount = 0; - while (latch.getCount() > 0 && changeLimitCount < nbMessages) { - changeLimitCount++; - int limit = progressiveLimitChange.applyAsInt(changeLimitCount); - int expectedMax = Math.min(10, limit); - limiter.setLimit(limit); - maxConcurrentRequest.set(0); - sleep(random.nextInt(20)); - int actualLimit = Math.min(10, limit); - int max = maxConcurrentRequest.get(); - if (max > 0) { - // Ignore iterations where nothing was polled (messages consumption slower than iteration) - limitsSum += actualLimit; - maxConcurrentRqSum += max; - } - eventsCsvWriter.registerEvent("max_concurrent_rq", max); - eventsCsvWriter.registerEvent("concurrent_rq", concurrentRequest.get()); - eventsCsvWriter.registerEvent("limit", limit); - eventsCsvWriter.registerEvent("in_flight", limiter.inFlight.get()); - eventsCsvWriter.registerEvent("expected_max", expectedMax); - eventsCsvWriter.registerEvent("max_minus_expected_max", max - expectedMax); - } - eventsCsvWriter.write(Path.of( - "target/0-stats-unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity.csv")); - assertThat(maxConcurrentRqSum).isLessThanOrEqualTo(limitsSum); - assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); - } - finally { - container.stop(); - } - } - - private static void sleep(long millis) { - try { - Thread.sleep(millis); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - static class ReceivesMessageListener { @Autowired From 0c1880118bde17295faeff855be3639a880b5aa0 Mon Sep 17 00:00:00 2001 From: Loic Rouchon Date: Wed, 12 Feb 2025 17:42:11 +0100 Subject: [PATCH 6/9] Add a wait condition to the CompositeBPH in case 0 permits were returned (#1251) The wait can be interrupted when permits are returned. --- .../CompositeBackPressureHandler.java | 54 +++++++- .../SqsBackPressureIntegrationTests.java | 131 ++++++------------ 2 files changed, 95 insertions(+), 90 deletions(-) diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java index 42202438b..92a1e5295 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java @@ -17,18 +17,33 @@ import java.time.Duration; import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class CompositeBackPressureHandler implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { + private static final Logger logger = LoggerFactory.getLogger(CompositeBackPressureHandler.class); + private final List backPressureHandlers; private final int batchSize; + private final ReentrantLock noPermitsReturnedWaitLock = new ReentrantLock(); + + private final Condition permitsReleasedCondition = noPermitsReturnedWaitLock.newCondition(); + + private final Duration noPermitsReturnedWaitTimeout; + private String id; - public CompositeBackPressureHandler(List backPressureHandlers, int batchSize) { + public CompositeBackPressureHandler(List backPressureHandlers, int batchSize, + Duration waitTimeout) { this.backPressureHandlers = backPressureHandlers; this.batchSize = batchSize; + this.noPermitsReturnedWaitTimeout = waitTimeout; } @Override @@ -63,6 +78,9 @@ public int request(int amount) throws InterruptedException { backPressureHandlers.get(i).release(obtainedForBph - obtained, ReleaseReason.LIMITED); } } + if (obtained == 0) { + waitForPermitsToBeReleased(); + } return obtained; } @@ -71,14 +89,48 @@ public void release(int amount, ReleaseReason reason) { for (BackPressureHandler handler : backPressureHandlers) { handler.release(amount, reason); } + if (amount > 0) { + signalPermitsWereReleased(); + } + } + + /** + * Waits for permits to be released up to {@link #noPermitsReturnedWaitTimeout}. If no permits were released within + * the configured {@link #noPermitsReturnedWaitTimeout}, returns immediately. This allows {@link #request(int)} to + * return {@code 0} permits and will trigger another round of back-pressure handling. + * + * @throws InterruptedException if the Thread is interrupted while waiting for permits. + */ + @SuppressWarnings({ "java:S899" // we are not interested in the await return value here + }) + private void waitForPermitsToBeReleased() throws InterruptedException { + noPermitsReturnedWaitLock.lock(); + try { + permitsReleasedCondition.await(noPermitsReturnedWaitTimeout.toMillis(), TimeUnit.MILLISECONDS); + } + finally { + noPermitsReturnedWaitLock.unlock(); + } + } + + private void signalPermitsWereReleased() { + noPermitsReturnedWaitLock.lock(); + try { + permitsReleasedCondition.signal(); + } + finally { + noPermitsReturnedWaitLock.unlock(); + } } @Override public boolean drain(Duration timeout) { + logger.info("Draining back-pressure handlers initiated"); boolean result = true; for (BackPressureHandler handler : backPressureHandlers) { result &= !handler.drain(timeout); } + logger.info("Draining back-pressure handlers completed"); return result; } } diff --git a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java index 7fc18e308..6decea11f 100644 --- a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java +++ b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java @@ -15,7 +15,6 @@ */ package io.awspring.cloud.sqs.integration; -import static java.util.Collections.singletonMap; import static org.assertj.core.api.Assertions.assertThat; import io.awspring.cloud.sqs.config.SqsBootstrapConfiguration; @@ -36,7 +35,6 @@ import java.util.List; import java.util.Queue; import java.util.Random; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; @@ -47,7 +45,6 @@ import java.util.function.IntUnaryOperator; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; @@ -60,8 +57,6 @@ import org.springframework.context.annotation.Import; import org.springframework.messaging.Message; import org.springframework.messaging.support.MessageBuilder; -import software.amazon.awssdk.services.sqs.SqsAsyncClient; -import software.amazon.awssdk.services.sqs.model.QueueAttributeName; /** * Integration tests for SQS containers back pressure management. @@ -73,65 +68,6 @@ class SqsBackPressureIntegrationTests extends BaseSqsIntegrationTest { private static final Logger logger = LoggerFactory.getLogger(SqsBackPressureIntegrationTests.class); - static final String RECEIVES_MESSAGE_QUEUE_NAME = "receives_message_test_queue"; - - static final String RECEIVES_MESSAGE_BATCH_QUEUE_NAME = "receives_message_batch_test_queue"; - - static final String RECEIVES_MESSAGE_ASYNC_QUEUE_NAME = "receives_message_async_test_queue"; - - static final String DOES_NOT_ACK_ON_ERROR_QUEUE_NAME = "does_not_ack_test_queue"; - - static final String DOES_NOT_ACK_ON_ERROR_ASYNC_QUEUE_NAME = "does_not_ack_async_test_queue"; - - static final String DOES_NOT_ACK_ON_ERROR_BATCH_QUEUE_NAME = "does_not_ack_batch_test_queue"; - - static final String DOES_NOT_ACK_ON_ERROR_BATCH_ASYNC_QUEUE_NAME = "does_not_ack_batch_async_test_queue"; - - static final String RESOLVES_PARAMETER_TYPES_QUEUE_NAME = "resolves_parameter_type_test_queue"; - - static final String MANUALLY_START_CONTAINER = "manually_start_container_test_queue"; - - static final String MANUALLY_CREATE_CONTAINER_QUEUE_NAME = "manually_create_container_test_queue"; - - static final String MANUALLY_CREATE_INACTIVE_CONTAINER_QUEUE_NAME = "manually_create_inactive_container_test_queue"; - - static final String MANUALLY_CREATE_FACTORY_QUEUE_NAME = "manually_create_factory_test_queue"; - - static final String CONSUMES_ONE_MESSAGE_AT_A_TIME_QUEUE_NAME = "consumes_one_message_test_queue"; - - static final String MAX_CONCURRENT_MESSAGES_QUEUE_NAME = "max_concurrent_messages_test_queue"; - - static final String LOW_RESOURCE_FACTORY = "lowResourceFactory"; - - static final String MANUAL_ACK_FACTORY = "manualAcknowledgementFactory"; - - static final String MANUAL_ACK_BATCH_FACTORY = "manualAcknowledgementBatchFactory"; - - static final String ACK_AFTER_SECOND_ERROR_FACTORY = "ackAfterSecondErrorFactory"; - - @BeforeAll - static void beforeTests() { - SqsAsyncClient client = createAsyncClient(); - CompletableFuture.allOf(createQueue(client, RECEIVES_MESSAGE_QUEUE_NAME), - createQueue(client, DOES_NOT_ACK_ON_ERROR_QUEUE_NAME, - singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "1")), - createQueue(client, DOES_NOT_ACK_ON_ERROR_ASYNC_QUEUE_NAME, - singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "1")), - createQueue(client, DOES_NOT_ACK_ON_ERROR_BATCH_QUEUE_NAME, - singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "1")), - createQueue(client, DOES_NOT_ACK_ON_ERROR_BATCH_ASYNC_QUEUE_NAME, - singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "1")), - createQueue(client, RECEIVES_MESSAGE_ASYNC_QUEUE_NAME), - createQueue(client, RECEIVES_MESSAGE_BATCH_QUEUE_NAME), - createQueue(client, RESOLVES_PARAMETER_TYPES_QUEUE_NAME, - singletonMap(QueueAttributeName.VISIBILITY_TIMEOUT, "20")), - createQueue(client, MANUALLY_CREATE_CONTAINER_QUEUE_NAME), - createQueue(client, MANUALLY_CREATE_INACTIVE_CONTAINER_QUEUE_NAME), - createQueue(client, MANUALLY_CREATE_FACTORY_QUEUE_NAME), - createQueue(client, CONSUMES_ONE_MESSAGE_AT_A_TIME_QUEUE_NAME), - createQueue(client, MAX_CONCURRENT_MESSAGES_QUEUE_NAME)).join(); - } - @Autowired SqsTemplate sqsTemplate; @@ -202,11 +138,12 @@ void staticBackPressureLimitShouldCapQueueProcessingCapacity(int staticLimit, in .queueNames( queueName) .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) - .acquireTimeout(Duration.ofSeconds(1L)) - .throughputConfiguration(BackPressureMode.AUTO).build()), - 5))) + .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler( + List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 5, Duration.ofMillis(50L)))) .messageListener(msg -> { int concurrentRqs = concurrentRequest.incrementAndGet(); maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); @@ -241,11 +178,12 @@ void zeroBackPressureLimitShouldStopQueueProcessing() throws Exception { .queueNames( queueName) .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) - .acquireTimeout(Duration.ofSeconds(1L)) - .throughputConfiguration(BackPressureMode.AUTO).build()), - 5))) + .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler( + List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 5, Duration.ofMillis(50L)))) .messageListener(msg -> { int concurrentRqs = concurrentRequest.incrementAndGet(); maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); @@ -278,23 +216,33 @@ void changeInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Except var latch = new CountDownLatch(nbMessages); var controlSemaphore = new Semaphore(0); var advanceSemaphore = new Semaphore(0); + var processingFailed = new AtomicBoolean(false); + var isDraining = new AtomicBoolean(false); var container = SqsMessageListenerContainer .builder().sqsAsyncClient( BaseSqsIntegrationTest.createAsyncClient()) .queueNames( queueName) .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) - .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler(List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) - .acquireTimeout(Duration.ofSeconds(1L)) - .throughputConfiguration(BackPressureMode.AUTO).build()), - 5))) + .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler( + List.of(limiter, + SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) + .acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 5, Duration.ofMillis(50L)))) .messageListener(msg -> { try { - controlSemaphore.acquire(); + if (!controlSemaphore.tryAcquire(5, TimeUnit.SECONDS) && !isDraining.get()) { + processingFailed.set(true); + throw new IllegalStateException("Failed to wait for control semaphore"); + } } catch (InterruptedException e) { - throw new RuntimeException(e); + if (!isDraining.get()) { + processingFailed.set(true); + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } } int concurrentRqs = concurrentRequest.incrementAndGet(); maxConcurrentRequest.updateAndGet(max -> Math.max(max, concurrentRqs)); @@ -310,14 +258,16 @@ class Controller { private final Semaphore controlSemaphore; private final NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter; private final AtomicInteger maxConcurrentRequest; + private final AtomicBoolean processingFailed; Controller(Semaphore advanceSemaphore, Semaphore controlSemaphore, NonBlockingExternalConcurrencyLimiterBackPressureHandler limiter, - AtomicInteger maxConcurrentRequest) { + AtomicInteger maxConcurrentRequest, AtomicBoolean processingFailed) { this.advanceSemaphore = advanceSemaphore; this.controlSemaphore = controlSemaphore; this.limiter = limiter; this.maxConcurrentRequest = maxConcurrentRequest; + this.processingFailed = processingFailed; } public void updateLimit(int newLimit) { @@ -341,9 +291,11 @@ void waitForAdvance(int permits) throws InterruptedException { .withFailMessage(() -> "Waiting for %d permits timed out. Only %d permits available" .formatted(permits, advanceSemaphore.availablePermits())) .isTrue(); + assertThat(processingFailed.get()).isFalse(); } } - var controller = new Controller(advanceSemaphore, controlSemaphore, limiter, maxConcurrentRequest); + var controller = new Controller(advanceSemaphore, controlSemaphore, limiter, maxConcurrentRequest, + processingFailed); try { container.start(); @@ -386,8 +338,10 @@ void waitForAdvance(int permits) throws InterruptedException { controller.waitForAdvance(50); assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); assertThat(controller.maxConcurrentRequest.get()).isEqualTo(5); + assertThat(processingFailed.get()).isFalse(); } finally { + isDraining.set(true); container.stop(); } } @@ -500,13 +454,12 @@ void unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity( options -> options.pollTimeout(Duration.ofSeconds(1)) .standbyLimitPollingInterval( Duration.ofMillis(1)) - .backPressureHandlerSupplier(() -> new StatisticsBphDecorator( - new CompositeBackPressureHandler(List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(10).totalPermits(10) - .acquireTimeout(Duration.ofSeconds(1L)) + .backPressureHandlerSupplier( + () -> new StatisticsBphDecorator(new CompositeBackPressureHandler( + List.of(limiter, SemaphoreBackPressureHandler.builder().batchSize(10) + .totalPermits(10).acquireTimeout(Duration.ofSeconds(1L)) .throughputConfiguration(BackPressureMode.AUTO).build()), - 10), - eventsCsvWriter))) + 10, Duration.ofMillis(50L)), eventsCsvWriter))) .messageListener(msg -> { int currentConcurrentRq = concurrentRequest.incrementAndGet(); maxConcurrentRequest.updateAndGet(max -> Math.max(max, currentConcurrentRq)); From 98ba703ba98cfcc648513c9755e0d85412cffaf5 Mon Sep 17 00:00:00 2001 From: Loic Rouchon Date: Thu, 13 Feb 2025 14:51:10 +0100 Subject: [PATCH 7/9] Enhance default methods for backward compatibility (#1251) --- .../io/awspring/cloud/sqs/listener/BackPressureHandler.java | 5 ++++- .../cloud/sqs/listener/BatchAwareBackPressureHandler.java | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java index a5921de68..55e5a25f0 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BackPressureHandler.java @@ -55,7 +55,9 @@ public interface BackPressureHandler { * @param amount the amount of permits to release. * @param reason the reason why the permits were released. */ - void release(int amount, ReleaseReason reason); + default void release(int amount, ReleaseReason reason) { + release(amount); + } /** * Release the specified amount of permits. Each message that has been processed should release one permit, whether @@ -67,6 +69,7 @@ public interface BackPressureHandler { */ @Deprecated default void release(int amount) { + release(amount, ReleaseReason.PROCESSED); } /** diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java index 06387976c..c5ccf0ba4 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/BatchAwareBackPressureHandler.java @@ -41,6 +41,7 @@ public interface BatchAwareBackPressureHandler extends BackPressureHandler { */ @Deprecated default void releaseBatch() { + release(getBatchSize(), ReleaseReason.NONE_FETCHED); } /** From 011fde0c46a04cd5d3ab833deb8e43a91f594d5d Mon Sep 17 00:00:00 2001 From: Loic Rouchon Date: Mon, 17 Feb 2025 14:06:49 +0100 Subject: [PATCH 8/9] Split SemaphoreBackPressureHandler into a ConcurrencyLimiterBlocking and a Throughput BackPressureHandler(s) (#1251) --- ...tractPipelineMessageListenerContainer.java | 18 +- .../CompositeBackPressureHandler.java | 19 +- ...ncyLimiterBlockingBackPressureHandler.java | 163 +++++++++++ .../SemaphoreBackPressureHandler.java | 252 ------------------ .../ThroughputBackPressureHandler.java | 154 +++++++++++ .../SqsBackPressureIntegrationTests.java | 45 ++-- .../AbstractPollingMessageSourceTests.java | 186 +++++++++---- 7 files changed, 499 insertions(+), 338 deletions(-) create mode 100644 spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ConcurrencyLimiterBlockingBackPressureHandler.java delete mode 100644 spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java create mode 100644 spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ThroughputBackPressureHandler.java diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java index 5dcd5dd86..bdedc35de 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java @@ -232,9 +232,23 @@ protected BackPressureHandler createBackPressureHandler() { } Duration acquireTimeout = containerOptions.getMaxDelayBetweenPolls(); int batchSize = containerOptions.getMaxMessagesPerPoll(); - return SemaphoreBackPressureHandler.builder().batchSize(batchSize) - .totalPermits(containerOptions.getMaxConcurrentMessages()).acquireTimeout(acquireTimeout) + int maxConcurrentMessages = containerOptions.getMaxConcurrentMessages(); + var concurrencyLimiterBlockingBackPressureHandler = ConcurrencyLimiterBlockingBackPressureHandler.builder() + .batchSize(batchSize).totalPermits(maxConcurrentMessages).acquireTimeout(acquireTimeout) .throughputConfiguration(containerOptions.getBackPressureMode()).build(); + if (maxConcurrentMessages == batchSize) { + return concurrencyLimiterBlockingBackPressureHandler; + } + return switch (containerOptions.getBackPressureMode()) { + case FIXED_HIGH_THROUGHPUT -> concurrencyLimiterBlockingBackPressureHandler; + case ALWAYS_POLL_MAX_MESSAGES, + AUTO -> { + var throughputBackPressureHandler = ThroughputBackPressureHandler.builder().batchSize(batchSize).build(); + yield new CompositeBackPressureHandler( + List.of(concurrencyLimiterBlockingBackPressureHandler, throughputBackPressureHandler), + batchSize, containerOptions.getStandbyLimitPollingInterval()); + } + }; } protected TaskExecutor createSourcesTaskExecutor() { diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java index 92a1e5295..930f7dc6e 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/CompositeBackPressureHandler.java @@ -16,6 +16,7 @@ package io.awspring.cloud.sqs.listener; import java.time.Duration; +import java.time.Instant; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; @@ -66,6 +67,7 @@ public int requestBatch() throws InterruptedException { @Override public int request(int amount) throws InterruptedException { + logger.debug("[{}] Requesting {} permits", this.id, amount); int obtained = amount; int[] obtainedPerBph = new int[backPressureHandlers.size()]; for (int i = 0; i < backPressureHandlers.size() && obtained > 0; i++) { @@ -81,11 +83,13 @@ public int request(int amount) throws InterruptedException { if (obtained == 0) { waitForPermitsToBeReleased(); } + logger.debug("[{}] Obtained {} permits ({} requested)", this.id, obtained, amount); return obtained; } @Override public void release(int amount, ReleaseReason reason) { + logger.debug("[{}] Releasing {} permits ({})", this.id, amount, reason); for (BackPressureHandler handler : backPressureHandlers) { handler.release(amount, reason); } @@ -106,6 +110,8 @@ public void release(int amount, ReleaseReason reason) { private void waitForPermitsToBeReleased() throws InterruptedException { noPermitsReturnedWaitLock.lock(); try { + logger.trace("[{}] No permits were obtained, waiting for a release up to {}", this.id, + noPermitsReturnedWaitTimeout); permitsReleasedCondition.await(noPermitsReturnedWaitTimeout.toMillis(), TimeUnit.MILLISECONDS); } finally { @@ -125,12 +131,19 @@ private void signalPermitsWereReleased() { @Override public boolean drain(Duration timeout) { - logger.info("Draining back-pressure handlers initiated"); + logger.debug("[{}] Draining back-pressure handlers initiated", this.id); boolean result = true; + Instant start = Instant.now(); for (BackPressureHandler handler : backPressureHandlers) { - result &= !handler.drain(timeout); + Duration remainingTimeout = maxDuration(timeout.minus(Duration.between(start, Instant.now())), + Duration.ZERO); + result &= handler.drain(remainingTimeout); } - logger.info("Draining back-pressure handlers completed"); + logger.debug("[{}] Draining back-pressure handlers completed", this.id); return result; } + + private static Duration maxDuration(Duration first, Duration second) { + return first.compareTo(second) > 0 ? first : second; + } } diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ConcurrencyLimiterBlockingBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ConcurrencyLimiterBlockingBackPressureHandler.java new file mode 100644 index 000000000..e389ba7c3 --- /dev/null +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ConcurrencyLimiterBlockingBackPressureHandler.java @@ -0,0 +1,163 @@ +/* + * Copyright 2013-2022 the original author or authors. + * + * Licensed 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 + * + * https://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 io.awspring.cloud.sqs.listener; + +import java.time.Duration; +import java.util.Arrays; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.util.Assert; + +/** + * {@link BackPressureHandler} implementation that uses a {@link Semaphore} for handling backpressure. + * + * @author Tomaz Fernandes + * @see io.awspring.cloud.sqs.listener.source.PollingMessageSource + * @since 3.0 + */ +public class ConcurrencyLimiterBlockingBackPressureHandler + implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { + + private static final Logger logger = LoggerFactory.getLogger(ConcurrencyLimiterBlockingBackPressureHandler.class); + + private final Semaphore semaphore; + + private final int batchSize; + + private final int totalPermits; + + private final Duration acquireTimeout; + + private final boolean alwaysPollMasMessages; + + private String id = getClass().getSimpleName(); + + private ConcurrencyLimiterBlockingBackPressureHandler(Builder builder) { + this.batchSize = builder.batchSize; + this.totalPermits = builder.totalPermits; + this.acquireTimeout = builder.acquireTimeout; + this.alwaysPollMasMessages = BackPressureMode.ALWAYS_POLL_MAX_MESSAGES.equals(builder.backPressureMode); + this.semaphore = new Semaphore(totalPermits); + logger.debug( + "ConcurrencyLimiterBlockingBackPressureHandler created with configuration " + + "totalPermits: {}, batchSize: {}, acquireTimeout: {}, an alwaysPollMasMessages: {}", + this.totalPermits, this.batchSize, this.acquireTimeout, this.alwaysPollMasMessages); + } + + public static Builder builder() { + return new Builder(); + } + + @Override + public void setId(String id) { + this.id = id; + } + + @Override + public String getId() { + return this.id; + } + + @Override + public int requestBatch() throws InterruptedException { + return request(this.batchSize); + } + + @Override + public int request(int amount) throws InterruptedException { + int acquiredPermits = tryAcquire(amount, this.acquireTimeout); + if (alwaysPollMasMessages || acquiredPermits > 0) { + return acquiredPermits; + } + int availablePermits = Math.min(this.semaphore.availablePermits(), amount); + if (availablePermits > 0) { + return tryAcquire(availablePermits, this.acquireTimeout); + } + return 0; + } + + private int tryAcquire(int amount, Duration duration) throws InterruptedException { + if (this.semaphore.tryAcquire(amount, duration.toMillis(), TimeUnit.MILLISECONDS)) { + logger.debug("[{}] Acquired {} permits ({} / {} available)", this.id, amount, + this.semaphore.availablePermits(), this.totalPermits); + return amount; + } + return 0; + } + + @Override + public void release(int amount, ReleaseReason reason) { + this.semaphore.release(amount); + logger.debug("[{}] Released {} permits ({}) ({} / {} available)", this.id, amount, reason, + this.semaphore.availablePermits(), this.totalPermits); + } + + @Override + public boolean drain(Duration timeout) { + logger.debug("[{}] Waiting for up to {} for approx. {} permits to be released", this.id, timeout, + this.totalPermits - this.semaphore.availablePermits()); + try { + return tryAcquire(this.totalPermits, timeout) > 0; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.debug("[{}] Draining interrupted", this.id); + return false; + } + } + + public static class Builder { + + private int batchSize; + + private int totalPermits; + + private Duration acquireTimeout; + + private BackPressureMode backPressureMode; + + public Builder batchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + public Builder totalPermits(int totalPermits) { + this.totalPermits = totalPermits; + return this; + } + + public Builder acquireTimeout(Duration acquireTimeout) { + this.acquireTimeout = acquireTimeout; + return this; + } + + public Builder throughputConfiguration(BackPressureMode backPressureConfiguration) { + this.backPressureMode = backPressureConfiguration; + return this; + } + + public ConcurrencyLimiterBlockingBackPressureHandler build() { + Assert.noNullElements( + Arrays.asList(this.batchSize, this.totalPermits, this.acquireTimeout, this.backPressureMode), + "Missing configuration"); + Assert.isTrue(this.batchSize > 0, "The batch size must be greater than 0"); + Assert.isTrue(this.totalPermits >= this.batchSize, "Total permits must be greater than the batch size"); + return new ConcurrencyLimiterBlockingBackPressureHandler(this); + } + } +} diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java deleted file mode 100644 index 70ed3f306..000000000 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java +++ /dev/null @@ -1,252 +0,0 @@ -/* - * Copyright 2013-2022 the original author or authors. - * - * Licensed 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 - * - * https://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 io.awspring.cloud.sqs.listener; - -import java.time.Duration; -import java.util.Arrays; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.util.Assert; - -/** - * {@link BackPressureHandler} implementation that uses a {@link Semaphore} for handling backpressure. - * - * @author Tomaz Fernandes - * @since 3.0 - * @see io.awspring.cloud.sqs.listener.source.PollingMessageSource - */ -public class SemaphoreBackPressureHandler implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { - - private static final Logger logger = LoggerFactory.getLogger(SemaphoreBackPressureHandler.class); - - private final Semaphore semaphore; - - private final int batchSize; - - private final int totalPermits; - - private final Duration acquireTimeout; - - private final BackPressureMode backPressureConfiguration; - - private volatile CurrentThroughputMode currentThroughputMode; - - private final AtomicInteger lowThroughputPermitsAcquired = new AtomicInteger(0); - - private String id; - - private SemaphoreBackPressureHandler(Builder builder) { - this.batchSize = builder.batchSize; - this.totalPermits = builder.totalPermits; - this.acquireTimeout = builder.acquireTimeout; - this.backPressureConfiguration = builder.backPressureMode; - this.semaphore = new Semaphore(totalPermits); - this.currentThroughputMode = BackPressureMode.FIXED_HIGH_THROUGHPUT.equals(backPressureConfiguration) - ? CurrentThroughputMode.HIGH - : CurrentThroughputMode.LOW; - logger.debug("SemaphoreBackPressureHandler created with configuration {} and {} total permits", - backPressureConfiguration, totalPermits); - } - - public static Builder builder() { - return new Builder(); - } - - @Override - public void setId(String id) { - this.id = id; - } - - @Override - public String getId() { - return this.id; - } - - @Override - public int requestBatch() throws InterruptedException { - return request(batchSize); - } - - // @formatter:off - @Override - public int request(int amount) throws InterruptedException { - return CurrentThroughputMode.LOW.equals(this.currentThroughputMode) - ? requestInLowThroughputMode(amount) - : requestInHighThroughputMode(amount); - } - - private int requestInHighThroughputMode(int amount) throws InterruptedException { - return tryAcquire(amount, CurrentThroughputMode.HIGH) - ? amount - : tryAcquirePartial(amount); - } - // @formatter:on - - private int tryAcquirePartial(int max) throws InterruptedException { - int availablePermits = this.semaphore.availablePermits(); - if (availablePermits == 0 || BackPressureMode.ALWAYS_POLL_MAX_MESSAGES.equals(this.backPressureConfiguration)) { - return 0; - } - int permitsToRequest = Math.min(availablePermits, max); - CurrentThroughputMode currentThroughputModeNow = this.currentThroughputMode; - logger.trace("Trying to acquire partial batch of {} permits from {} available for {} in TM {}", - permitsToRequest, availablePermits, this.id, currentThroughputModeNow); - boolean hasAcquiredPartial = tryAcquire(permitsToRequest, currentThroughputModeNow); - return hasAcquiredPartial ? permitsToRequest : 0; - } - - private int requestInLowThroughputMode(int amount) throws InterruptedException { - // Although LTM can be set / unset by many processes, only the MessageSource thread gets here, - // so no actual concurrency - logger.debug("Trying to acquire full permits for {}. Permits left: {}", this.id, - this.semaphore.availablePermits()); - boolean hasAcquired = tryAcquire(this.totalPermits, CurrentThroughputMode.LOW); - if (hasAcquired) { - logger.debug("Acquired full permits for {}. Permits left: {}", this.id, this.semaphore.availablePermits()); - // We've acquired all permits - there's no other process currently processing messages - if (this.lowThroughputPermitsAcquired.getAndSet(amount) != 0) { - logger.warn("hasAcquiredFullPermits was already true. Permits left: {}", - this.semaphore.availablePermits()); - } - return amount; - } - else { - return 0; - } - } - - private boolean tryAcquire(int amount, CurrentThroughputMode currentThroughputModeNow) throws InterruptedException { - logger.trace("Acquiring {} permits for {} in TM {}", amount, this.id, this.currentThroughputMode); - boolean hasAcquired = this.semaphore.tryAcquire(amount, this.acquireTimeout.toMillis(), TimeUnit.MILLISECONDS); - if (hasAcquired) { - logger.trace("{} permits acquired for {} in TM {}. Permits left: {}", amount, this.id, - currentThroughputModeNow, this.semaphore.availablePermits()); - } - else { - logger.trace("Not able to acquire {} permits in {} milliseconds for {} in TM {}. Permits left: {}", amount, - this.acquireTimeout.toMillis(), this.id, currentThroughputModeNow, - this.semaphore.availablePermits()); - } - return hasAcquired; - } - - @Override - public void release(int amount, ReleaseReason reason) { - logger.trace("Releasing {} permits ({}) for {}. Permits left: {}", amount, reason, this.id, - this.semaphore.availablePermits()); - switch (reason) { - case NONE_FETCHED -> maybeSwitchToLowThroughputMode(); - case PARTIAL_FETCH -> maybeSwitchToHighThroughputMode(amount); - case PROCESSED, LIMITED -> { - // No need to switch throughput mode - } - } - int permitsToRelease = getPermitsToRelease(amount); - this.semaphore.release(permitsToRelease); - logger.debug("Released {} permits ({}) for {}. Permits left: {}", permitsToRelease, reason, this.id, - this.semaphore.availablePermits()); - } - - private void maybeSwitchToLowThroughputMode() { - if (!BackPressureMode.FIXED_HIGH_THROUGHPUT.equals(this.backPressureConfiguration) - && CurrentThroughputMode.HIGH.equals(this.currentThroughputMode)) { - logger.debug("Entire batch of permits released for {}, setting TM LOW. Permits left: {}", this.id, - this.semaphore.availablePermits()); - this.currentThroughputMode = CurrentThroughputMode.LOW; - } - } - - private void maybeSwitchToHighThroughputMode(int amount) { - if (CurrentThroughputMode.LOW.equals(this.currentThroughputMode)) { - logger.debug("{} unused permit(s), setting TM HIGH for {}. Permits left: {}", amount, this.id, - this.semaphore.availablePermits()); - this.currentThroughputMode = CurrentThroughputMode.HIGH; - } - } - - private int getPermitsToRelease(int amount) { - int lowThroughputPermits = this.lowThroughputPermitsAcquired.getAndSet(0); - return lowThroughputPermits > 0 - // The first process that gets here should release all permits except for inflight messages - // We can have only one batch of messages at this point since we have all permits - ? this.totalPermits - (lowThroughputPermits - amount) - : amount; - } - - @Override - public boolean drain(Duration timeout) { - logger.debug("Waiting for up to {} seconds for approx. {} permits to be released for {}", timeout.getSeconds(), - this.totalPermits - this.semaphore.availablePermits(), this.id); - try { - return this.semaphore.tryAcquire(this.totalPermits, (int) timeout.getSeconds(), TimeUnit.SECONDS); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IllegalStateException("Interrupted while waiting to acquire permits", e); - } - } - - private enum CurrentThroughputMode { - - HIGH, - - LOW; - - } - - public static class Builder { - - private int batchSize; - - private int totalPermits; - - private Duration acquireTimeout; - - private BackPressureMode backPressureMode; - - public Builder batchSize(int batchSize) { - this.batchSize = batchSize; - return this; - } - - public Builder totalPermits(int totalPermits) { - this.totalPermits = totalPermits; - return this; - } - - public Builder acquireTimeout(Duration acquireTimeout) { - this.acquireTimeout = acquireTimeout; - return this; - } - - public Builder throughputConfiguration(BackPressureMode backPressureConfiguration) { - this.backPressureMode = backPressureConfiguration; - return this; - } - - public SemaphoreBackPressureHandler build() { - Assert.noNullElements( - Arrays.asList(this.batchSize, this.totalPermits, this.acquireTimeout, this.backPressureMode), - "Missing configuration"); - return new SemaphoreBackPressureHandler(this); - } - - } - -} diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ThroughputBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ThroughputBackPressureHandler.java new file mode 100644 index 000000000..3ef1410d9 --- /dev/null +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ThroughputBackPressureHandler.java @@ -0,0 +1,154 @@ +/* + * Copyright 2013-2022 the original author or authors. + * + * Licensed 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 + * + * https://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 io.awspring.cloud.sqs.listener; + +import io.awspring.cloud.sqs.listener.source.PollingMessageSource; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.util.Assert; + +/** + * {@link BackPressureHandler} implementation that uses a switches between high and low throughput modes. + *

+ * The initial throughput mode is low, which means, only one batch at a time can be requested. If some messages are + * fetched, then the throughput mode is switched to high, which means, the multiple batches can be requested (i.e. there + * is no need to wait for the previous batch's processing to complete before requesting a new one). If no messages are + * returned fetched by a poll, the throughput mode is switched back to low. + *

+ * This {@link BackPressureHandler} is designed to be used in combination with another {@link BackPressureHandler} like + * the {@link ConcurrencyLimiterBlockingBackPressureHandler} that will handle the maximum concurrency level within the + * application. + * + * @author Tomaz Fernandes + * @see PollingMessageSource + * @since 3.0 + */ +public class ThroughputBackPressureHandler implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { + + private static final Logger logger = LoggerFactory.getLogger(ThroughputBackPressureHandler.class); + + private final int batchSize; + + private final AtomicReference currentThroughputMode = new AtomicReference<>( + CurrentThroughputMode.LOW); + + private final AtomicInteger inFlightRequests = new AtomicInteger(0); + + private final AtomicBoolean drained = new AtomicBoolean(false); + + private String id = getClass().getSimpleName(); + + private ThroughputBackPressureHandler(Builder builder) { + this.batchSize = builder.batchSize; + logger.debug("ThroughputBackPressureHandler created with batchSize {}", this.batchSize); + } + + public static Builder builder() { + return new Builder(); + } + + @Override + public void setId(String id) { + this.id = id; + } + + @Override + public String getId() { + return this.id; + } + + @Override + public int requestBatch() throws InterruptedException { + return request(this.batchSize); + } + + @Override + public int request(int amount) throws InterruptedException { + if (drained.get()) { + return 0; + } + int permits; + int inFlight = inFlightRequests.get(); + if (CurrentThroughputMode.LOW == this.currentThroughputMode.get()) { + permits = Math.max(0, Math.min(amount, this.batchSize - inFlight)); + logger.debug("[{}] Acquired {} permits (low throughput mode), in flight: {}", this.id, amount, inFlight); + } + else { + permits = amount; + logger.debug("[{}] Acquired {} permits (high throughput mode), in flight: {}", this.id, amount, inFlight); + } + inFlightRequests.addAndGet(permits); + return permits; + } + + @Override + public void release(int amount, ReleaseReason reason) { + if (drained.get()) { + return; + } + logger.debug("[{}] Releasing {} permits ({})", this.id, amount, reason); + inFlightRequests.addAndGet(-amount); + switch (reason) { + case NONE_FETCHED -> updateThroughputMode(CurrentThroughputMode.HIGH, CurrentThroughputMode.LOW); + case PARTIAL_FETCH -> updateThroughputMode(CurrentThroughputMode.LOW, CurrentThroughputMode.HIGH); + case LIMITED, PROCESSED -> { + // No need to switch throughput mode + } + } + } + + private void updateThroughputMode(CurrentThroughputMode currentTarget, CurrentThroughputMode newTarget) { + if (this.currentThroughputMode.compareAndSet(currentTarget, newTarget)) { + logger.debug("[{}] throughput mode updated to {}", this.id, newTarget); + } + } + + @Override + public boolean drain(Duration timeout) { + logger.debug("[{}] Draining", this.id); + drained.set(true); + return true; + } + + private enum CurrentThroughputMode { + + HIGH, + + LOW; + + } + + public static class Builder { + + private int batchSize; + + public Builder batchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + public ThroughputBackPressureHandler build() { + Assert.noNullElements(List.of(this.batchSize), "Missing configuration"); + Assert.isTrue(this.batchSize > 0, "batch size must be greater than 0"); + return new ThroughputBackPressureHandler(this); + } + } +} diff --git a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java index 6decea11f..8038f70d2 100644 --- a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java +++ b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/integration/SqsBackPressureIntegrationTests.java @@ -18,13 +18,7 @@ import static org.assertj.core.api.Assertions.assertThat; import io.awspring.cloud.sqs.config.SqsBootstrapConfiguration; -import io.awspring.cloud.sqs.listener.BackPressureHandler; -import io.awspring.cloud.sqs.listener.BackPressureMode; -import io.awspring.cloud.sqs.listener.BatchAwareBackPressureHandler; -import io.awspring.cloud.sqs.listener.CompositeBackPressureHandler; -import io.awspring.cloud.sqs.listener.IdentifiableContainerComponent; -import io.awspring.cloud.sqs.listener.SemaphoreBackPressureHandler; -import io.awspring.cloud.sqs.listener.SqsMessageListenerContainer; +import io.awspring.cloud.sqs.listener.*; import io.awspring.cloud.sqs.operations.SqsTemplate; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -140,8 +134,8 @@ void staticBackPressureLimitShouldCapQueueProcessingCapacity(int staticLimit, in .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler( List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) - .acquireTimeout(Duration.ofSeconds(1L)) + ConcurrencyLimiterBlockingBackPressureHandler.builder().batchSize(5) + .totalPermits(5).acquireTimeout(Duration.ofSeconds(1L)) .throughputConfiguration(BackPressureMode.AUTO).build()), 5, Duration.ofMillis(50L)))) .messageListener(msg -> { @@ -180,8 +174,8 @@ void zeroBackPressureLimitShouldStopQueueProcessing() throws Exception { .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler( List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) - .acquireTimeout(Duration.ofSeconds(1L)) + ConcurrencyLimiterBlockingBackPressureHandler.builder().batchSize(5) + .totalPermits(5).acquireTimeout(Duration.ofSeconds(1L)) .throughputConfiguration(BackPressureMode.AUTO).build()), 5, Duration.ofMillis(50L)))) .messageListener(msg -> { @@ -226,8 +220,8 @@ void changeInBackPressureLimitShouldAdaptQueueProcessingCapacity() throws Except .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) .backPressureHandlerSupplier(() -> new CompositeBackPressureHandler( List.of(limiter, - SemaphoreBackPressureHandler.builder().batchSize(5).totalPermits(5) - .acquireTimeout(Duration.ofSeconds(1L)) + ConcurrencyLimiterBlockingBackPressureHandler.builder().batchSize(5) + .totalPermits(5).acquireTimeout(Duration.ofSeconds(1L)) .throughputConfiguration(BackPressureMode.AUTO).build()), 5, Duration.ofMillis(50L)))) .messageListener(msg -> { @@ -446,20 +440,16 @@ void unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity( logger.debug("Sent {} messages to queue {}", nbMessages, queueName); var latch = new CountDownLatch(nbMessages); EventsCsvWriter eventsCsvWriter = new EventsCsvWriter(); - var container = SqsMessageListenerContainer - .builder().sqsAsyncClient( - BaseSqsIntegrationTest.createAsyncClient()) + var container = SqsMessageListenerContainer.builder().sqsAsyncClient(BaseSqsIntegrationTest.createAsyncClient()) .queueNames(queueName) - .configure( - options -> options.pollTimeout(Duration.ofSeconds(1)) - .standbyLimitPollingInterval( - Duration.ofMillis(1)) - .backPressureHandlerSupplier( - () -> new StatisticsBphDecorator(new CompositeBackPressureHandler( - List.of(limiter, SemaphoreBackPressureHandler.builder().batchSize(10) - .totalPermits(10).acquireTimeout(Duration.ofSeconds(1L)) - .throughputConfiguration(BackPressureMode.AUTO).build()), - 10, Duration.ofMillis(50L)), eventsCsvWriter))) + .configure(options -> options.pollTimeout(Duration.ofSeconds(1)) + .standbyLimitPollingInterval(Duration.ofMillis(1)) + .backPressureHandlerSupplier(() -> new StatisticsBphDecorator(new CompositeBackPressureHandler( + List.of(limiter, + ConcurrencyLimiterBlockingBackPressureHandler.builder().batchSize(10) + .totalPermits(10).acquireTimeout(Duration.ofSeconds(1L)) + .throughputConfiguration(BackPressureMode.AUTO).build()), + 10, Duration.ofMillis(50L)), eventsCsvWriter))) .messageListener(msg -> { int currentConcurrentRq = concurrentRequest.incrementAndGet(); maxConcurrentRequest.updateAndGet(max -> Math.max(max, currentConcurrentRq)); @@ -507,8 +497,7 @@ void unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity( eventsCsvWriter.registerEvent("expected_max", expectedMax); eventsCsvWriter.registerEvent("max_minus_expected_max", max - expectedMax); } - eventsCsvWriter.write(Path.of( - "target/0-stats-unsynchronizedChangesInBackPressureLimitShouldAdaptQueueProcessingCapacity.csv")); + eventsCsvWriter.write(Path.of("target/stats-%s.csv".formatted(queueName))); assertThat(maxConcurrentRqSum).isLessThanOrEqualTo(limitsSum); assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); } diff --git a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSourceTests.java b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSourceTests.java index b03b308c6..0d83aca27 100644 --- a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSourceTests.java +++ b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSourceTests.java @@ -23,27 +23,17 @@ import static org.mockito.Mockito.times; import io.awspring.cloud.sqs.MessageExecutionThreadFactory; -import io.awspring.cloud.sqs.listener.BackPressureMode; -import io.awspring.cloud.sqs.listener.SemaphoreBackPressureHandler; -import io.awspring.cloud.sqs.listener.SqsContainerOptions; +import io.awspring.cloud.sqs.listener.*; import io.awspring.cloud.sqs.listener.acknowledgement.AcknowledgementCallback; import io.awspring.cloud.sqs.listener.acknowledgement.AcknowledgementProcessor; import io.awspring.cloud.sqs.support.converter.MessageConversionContext; import io.awspring.cloud.sqs.support.converter.SqsMessagingMessageConverter; import java.time.Duration; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Semaphore; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; +import java.util.*; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.IntStream; import org.assertj.core.api.InstanceOfAssertFactories; import org.awaitility.Awaitility; @@ -69,13 +59,77 @@ class AbstractPollingMessageSourceTests { void shouldAcquireAndReleaseFullPermits() { String testName = "shouldAcquireAndReleaseFullPermits"; - SemaphoreBackPressureHandler backPressureHandler = SemaphoreBackPressureHandler.builder() + BackPressureHandler backPressureHandler = ConcurrencyLimiterBlockingBackPressureHandler.builder() .acquireTimeout(Duration.ofMillis(200)).batchSize(10).totalPermits(10) .throughputConfiguration(BackPressureMode.ALWAYS_POLL_MAX_MESSAGES).build(); ExecutorService threadPool = Executors.newCachedThreadPool(); CountDownLatch pollingCounter = new CountDownLatch(3); CountDownLatch processingCounter = new CountDownLatch(1); + AbstractPollingMessageSource source = new AbstractPollingMessageSource<>() { + + private final AtomicBoolean hasReceived = new AtomicBoolean(false); + + @Override + protected CompletableFuture> doPollForMessages(int messagesToRequest) { + return CompletableFuture.supplyAsync(() -> { + try { + // Since BackPressureMode.ALWAYS_POLL_MAX_MESSAGES, should always be 10. + assertThat(messagesToRequest).isEqualTo(10); + assertAvailablePermits(backPressureHandler, 0); + boolean firstPoll = hasReceived.compareAndSet(false, true); + return firstPoll + ? (Collection) List.of(Message.builder() + .messageId(UUID.randomUUID().toString()).body("message").build()) + : Collections. emptyList(); + } + catch (Throwable t) { + logger.error("Error", t); + throw new RuntimeException(t); + } + }, threadPool).whenComplete((v, t) -> { + if (t == null) { + pollingCounter.countDown(); + } + }); + } + }; + + source.setBackPressureHandler(backPressureHandler); + source.setMessageSink((msgs, context) -> { + assertAvailablePermits(backPressureHandler, 9); + msgs.forEach(msg -> context.runBackPressureReleaseCallback()); + return CompletableFuture.runAsync(processingCounter::countDown); + }); + + source.setId(testName + " source"); + source.configure(SqsContainerOptions.builder().build()); + source.setTaskExecutor(createTaskExecutor(testName)); + source.setAcknowledgementProcessor(getNoOpsAcknowledgementProcessor()); + source.start(); + assertThat(doAwait(pollingCounter)).isTrue(); + assertThat(doAwait(processingCounter)).isTrue(); + } + + @Test + void shouldAdaptThroughputMode() { + String testName = "shouldAdaptThroughputMode"; + + int totalPermits = 20; + int batchSize = 10; + var concurrencyLimiterBlockingBackPressureHandler = ConcurrencyLimiterBlockingBackPressureHandler.builder() + .batchSize(batchSize).totalPermits(totalPermits) + .throughputConfiguration(BackPressureMode.ALWAYS_POLL_MAX_MESSAGES) + .acquireTimeout(Duration.ofSeconds(5L)).build(); + var throughputBackPressureHandler = ThroughputBackPressureHandler.builder().batchSize(batchSize).build(); + var backPressureHandler = new CompositeBackPressureHandler( + List.of(concurrencyLimiterBlockingBackPressureHandler, throughputBackPressureHandler), batchSize, + Duration.ofMillis(100L)); + ExecutorService threadPool = Executors.newCachedThreadPool(); + CountDownLatch pollingCounter = new CountDownLatch(3); + CountDownLatch processingCounter = new CountDownLatch(1); + Collection errors = new ConcurrentLinkedQueue<>(); + AbstractPollingMessageSource source = new AbstractPollingMessageSource<>() { private final AtomicBoolean hasReceived = new AtomicBoolean(false); @@ -88,20 +142,20 @@ protected CompletableFuture> doPollForMessages(int messagesT try { // Since BackPressureMode.ALWAYS_POLL_MAX_MESSAGES, should always be 10. assertThat(messagesToRequest).isEqualTo(10); - assertAvailablePermits(backPressureHandler, 0); + // assertAvailablePermits(backPressureHandler, 10); boolean firstPoll = hasReceived.compareAndSet(false, true); if (firstPoll) { - logger.debug("First poll"); + logger.warn("First poll"); // No permits released yet, should be TM low assertThroughputMode(backPressureHandler, "low"); } else if (hasMadeSecondPoll.compareAndSet(false, true)) { - logger.debug("Second poll"); + logger.warn("Second poll"); // Permits returned, should be high assertThroughputMode(backPressureHandler, "high"); } else { - logger.debug("Third poll"); + logger.warn("Third poll"); // Already returned full permits, should be low assertThroughputMode(backPressureHandler, "low"); } @@ -111,20 +165,24 @@ else if (hasMadeSecondPoll.compareAndSet(false, true)) { : Collections. emptyList(); } catch (Throwable t) { - logger.error("Error", t); + logger.error("Error (not expecting it)", t); throw new RuntimeException(t); } }, threadPool).whenComplete((v, t) -> { if (t == null) { + logger.warn("pas boom", t); pollingCounter.countDown(); } + else { + logger.warn("BOOOOOOOM", t); + errors.add(t); + } }); } }; source.setBackPressureHandler(backPressureHandler); source.setMessageSink((msgs, context) -> { - assertAvailablePermits(backPressureHandler, 9); msgs.forEach(msg -> context.runBackPressureReleaseCallback()); return CompletableFuture.runAsync(processingCounter::countDown); }); @@ -133,9 +191,16 @@ else if (hasMadeSecondPoll.compareAndSet(false, true)) { source.configure(SqsContainerOptions.builder().build()); source.setTaskExecutor(createTaskExecutor(testName)); source.setAcknowledgementProcessor(getNoOpsAcknowledgementProcessor()); - source.start(); - assertThat(doAwait(pollingCounter)).isTrue(); - assertThat(doAwait(processingCounter)).isTrue(); + try { + logger.warn("Yolo, let's start"); + source.start(); + assertThat(doAwait(pollingCounter)).isTrue(); + assertThat(doAwait(processingCounter)).isTrue(); + assertThat(errors).isEmpty(); + } + finally { + source.stop(); + } } private static final AtomicInteger testCounter = new AtomicInteger(); @@ -143,8 +208,8 @@ else if (hasMadeSecondPoll.compareAndSet(false, true)) { @Test void shouldAcquireAndReleasePartialPermits() { String testName = "shouldAcquireAndReleasePartialPermits"; - SemaphoreBackPressureHandler backPressureHandler = SemaphoreBackPressureHandler.builder() - .acquireTimeout(Duration.ofMillis(150)).batchSize(10).totalPermits(10) + ConcurrencyLimiterBlockingBackPressureHandler backPressureHandler = ConcurrencyLimiterBlockingBackPressureHandler + .builder().acquireTimeout(Duration.ofMillis(150)).batchSize(10).totalPermits(10) .throughputConfiguration(BackPressureMode.AUTO).build(); ExecutorService threadPool = Executors .newCachedThreadPool(new MessageExecutionThreadFactory("test " + testCounter.incrementAndGet())); @@ -159,8 +224,6 @@ void shouldAcquireAndReleasePartialPermits() { private final AtomicBoolean hasAcquired9 = new AtomicBoolean(false); - private final AtomicBoolean hasMadeThirdPoll = new AtomicBoolean(false); - @Override protected CompletableFuture> doPollForMessages(int messagesToRequest) { return CompletableFuture.supplyAsync(() -> { @@ -176,31 +239,20 @@ protected CompletableFuture> doPollForMessages(int messagesT assertThat(messagesToRequest).isEqualTo(10); assertAvailablePermits(backPressureHandler, 0); // No permits have been released yet - assertThroughputMode(backPressureHandler, "low"); } else if (hasAcquired9.compareAndSet(false, true)) { // Second poll, should have 9 logger.debug("Second poll - should request 9 messages"); assertThat(messagesToRequest).isEqualTo(9); assertAvailablePermitsLessThanOrEqualTo(backPressureHandler, 1); - // Has released 9 permits, should be TM HIGH - assertThroughputMode(backPressureHandler, "high"); + // Has released 9 permits processingLatch.countDown(); // Release processing now } else { - boolean thirdPoll = hasMadeThirdPoll.compareAndSet(false, true); // Third poll or later, should have 10 again logger.debug("Third poll - should request 10 messages"); assertThat(messagesToRequest).isEqualTo(10); assertAvailablePermits(backPressureHandler, 0); - if (thirdPoll) { - // Hasn't yet returned a full batch, should be TM High - assertThroughputMode(backPressureHandler, "high"); - } - else { - // Has returned all permits in third poll - assertThroughputMode(backPressureHandler, "low"); - } } if (shouldReturnMessage) { logger.debug("shouldReturnMessage, returning one message"); @@ -241,8 +293,8 @@ else if (hasAcquired9.compareAndSet(false, true)) { @Test void shouldReleasePermitsOnConversionErrors() { String testName = "shouldReleasePermitsOnConversionErrors"; - SemaphoreBackPressureHandler backPressureHandler = SemaphoreBackPressureHandler.builder() - .acquireTimeout(Duration.ofMillis(150)).batchSize(10).totalPermits(10) + ConcurrencyLimiterBlockingBackPressureHandler backPressureHandler = ConcurrencyLimiterBlockingBackPressureHandler + .builder().acquireTimeout(Duration.ofMillis(150)).batchSize(10).totalPermits(10) .throughputConfiguration(BackPressureMode.AUTO).build(); AtomicInteger convertedMessages = new AtomicInteger(0); @@ -304,9 +356,16 @@ void shouldBackOffIfPollingThrowsAnError() { var testName = "shouldBackOffIfPollingThrowsAnError"; - var backPressureHandler = SemaphoreBackPressureHandler.builder().acquireTimeout(Duration.ofMillis(200)) - .batchSize(10).totalPermits(40).throughputConfiguration(BackPressureMode.ALWAYS_POLL_MAX_MESSAGES) - .build(); + int totalPermits = 40; + int batchSize = 10; + var concurrencyLimiterBlockingBackPressureHandler = ConcurrencyLimiterBlockingBackPressureHandler.builder() + .batchSize(batchSize).totalPermits(totalPermits) + .throughputConfiguration(BackPressureMode.ALWAYS_POLL_MAX_MESSAGES) + .acquireTimeout(Duration.ofMillis(200)).build(); + var throughputBackPressureHandler = ThroughputBackPressureHandler.builder().batchSize(batchSize).build(); + var backPressureHandler = new CompositeBackPressureHandler( + List.of(concurrencyLimiterBlockingBackPressureHandler, throughputBackPressureHandler), batchSize, + Duration.ofSeconds(5L)); var currentPoll = new AtomicInteger(0); var waitThirdPollLatch = new CountDownLatch(4); @@ -363,24 +422,45 @@ private static boolean doAwait(CountDownLatch processingLatch) { } } - private void assertThroughputMode(SemaphoreBackPressureHandler backPressureHandler, String expectedThroughputMode) { - assertThat(ReflectionTestUtils.getField(backPressureHandler, "currentThroughputMode")) - .extracting(Object::toString).extracting(String::toLowerCase) + private void assertThroughputMode(BackPressureHandler backPressureHandler, String expectedThroughputMode) { + var bph = extractBackPressureHandler(backPressureHandler, ThroughputBackPressureHandler.class); + assertThat(getThroughputModeValue(bph, "currentThroughputMode")) .isEqualTo(expectedThroughputMode.toLowerCase()); } - private void assertAvailablePermits(SemaphoreBackPressureHandler backPressureHandler, int expectedPermits) { - assertThat(ReflectionTestUtils.getField(backPressureHandler, "semaphore")).asInstanceOf(type(Semaphore.class)) + private static String getThroughputModeValue(ThroughputBackPressureHandler bph, String targetThroughputMode) { + return ((AtomicReference) ReflectionTestUtils.getField(bph, targetThroughputMode)).get().toString() + .toLowerCase(Locale.ROOT); + } + + private void assertAvailablePermits(BackPressureHandler backPressureHandler, int expectedPermits) { + var bph = extractBackPressureHandler(backPressureHandler, ConcurrencyLimiterBlockingBackPressureHandler.class); + assertThat(ReflectionTestUtils.getField(bph, "semaphore")).asInstanceOf(type(Semaphore.class)) .extracting(Semaphore::availablePermits).isEqualTo(expectedPermits); } - private void assertAvailablePermitsLessThanOrEqualTo(SemaphoreBackPressureHandler backPressureHandler, - int maxExpectedPermits) { - assertThat(ReflectionTestUtils.getField(backPressureHandler, "semaphore")).asInstanceOf(type(Semaphore.class)) + private void assertAvailablePermitsLessThanOrEqualTo( + ConcurrencyLimiterBlockingBackPressureHandler backPressureHandler, int maxExpectedPermits) { + var bph = extractBackPressureHandler(backPressureHandler, ConcurrencyLimiterBlockingBackPressureHandler.class); + assertThat(ReflectionTestUtils.getField(bph, "semaphore")).asInstanceOf(type(Semaphore.class)) .extracting(Semaphore::availablePermits).asInstanceOf(InstanceOfAssertFactories.INTEGER) .isLessThanOrEqualTo(maxExpectedPermits); } + private T extractBackPressureHandler(BackPressureHandler bph, Class type) { + if (type.isInstance(bph)) { + return type.cast(bph); + } + if (bph instanceof CompositeBackPressureHandler cbph) { + List backPressureHandlers = (List) ReflectionTestUtils + .getField(cbph, "backPressureHandlers"); + return extractBackPressureHandler( + backPressureHandlers.stream().filter(type::isInstance).map(type::cast).findFirst().orElseThrow(), + type); + } + throw new NoSuchElementException("%s not found in %s".formatted(type.getSimpleName(), bph)); + } + // Used to slow down tests while developing private void doSleep(int time) { try { From 6c41b8655c6cbb8143bbbe7136484c5ea32dbbfc Mon Sep 17 00:00:00 2001 From: Loic Rouchon Date: Mon, 3 Mar 2025 15:58:38 +0100 Subject: [PATCH 9/9] Revert changes to SemaphoreBackPressureHandler not to change default behavior (#1251) --- ...tractPipelineMessageListenerContainer.java | 24 +- .../sqs/listener/ContainerOptionsBuilder.java | 56 +++- .../SemaphoreBackPressureHandler.java | 269 ++++++++++++++++++ .../AbstractPollingMessageSourceTests.java | 1 + 4 files changed, 329 insertions(+), 21 deletions(-) create mode 100644 spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java index bdedc35de..6d7a2637a 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/AbstractPipelineMessageListenerContainer.java @@ -35,7 +35,6 @@ import io.awspring.cloud.sqs.listener.source.AcknowledgementProcessingMessageSource; import io.awspring.cloud.sqs.listener.source.MessageSource; import io.awspring.cloud.sqs.listener.source.PollingMessageSource; -import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -230,25 +229,10 @@ protected BackPressureHandler createBackPressureHandler() { if (containerOptions.getBackPressureHandlerSupplier() != null) { return containerOptions.getBackPressureHandlerSupplier().get(); } - Duration acquireTimeout = containerOptions.getMaxDelayBetweenPolls(); - int batchSize = containerOptions.getMaxMessagesPerPoll(); - int maxConcurrentMessages = containerOptions.getMaxConcurrentMessages(); - var concurrencyLimiterBlockingBackPressureHandler = ConcurrencyLimiterBlockingBackPressureHandler.builder() - .batchSize(batchSize).totalPermits(maxConcurrentMessages).acquireTimeout(acquireTimeout) - .throughputConfiguration(containerOptions.getBackPressureMode()).build(); - if (maxConcurrentMessages == batchSize) { - return concurrencyLimiterBlockingBackPressureHandler; - } - return switch (containerOptions.getBackPressureMode()) { - case FIXED_HIGH_THROUGHPUT -> concurrencyLimiterBlockingBackPressureHandler; - case ALWAYS_POLL_MAX_MESSAGES, - AUTO -> { - var throughputBackPressureHandler = ThroughputBackPressureHandler.builder().batchSize(batchSize).build(); - yield new CompositeBackPressureHandler( - List.of(concurrencyLimiterBlockingBackPressureHandler, throughputBackPressureHandler), - batchSize, containerOptions.getStandbyLimitPollingInterval()); - } - }; + return SemaphoreBackPressureHandler.builder().batchSize(getContainerOptions().getMaxMessagesPerPoll()) + .totalPermits(getContainerOptions().getMaxConcurrentMessages()) + .acquireTimeout(getContainerOptions().getMaxDelayBetweenPolls()) + .throughputConfiguration(getContainerOptions().getBackPressureMode()).build(); } protected TaskExecutor createSourcesTaskExecutor() { diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java index cf8c768d3..161687b6c 100644 --- a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/ContainerOptionsBuilder.java @@ -156,7 +156,61 @@ default B pollBackOffPolicy(BackOffPolicy pollBackOffPolicy) { B backPressureMode(BackPressureMode backPressureMode); /** - * Set the {@link Supplier} of {@link BackPressureHandler} for this container. Default is {@code null}. + * Sets the {@link Supplier} of {@link BackPressureHandler} for this container. Default is {@code null} which + * results in a default {@link SemaphoreBackPressureHandler} to be instantiated. In case a supplier is provided, the + * {@link BackPressureHandler} will be instantiated by the supplier. + *

+ * NOTE: it is important for the supplier to always return a new instance as otherwise it might + * result in a BackPressureHandler internal resources (counters, semaphores, ...) to be shared by multiple + * containers which is very likely not the desired behavior. + *

+ * Spring Cloud AWS provides the following {@link BackPressureHandler} implementations: + *

    + *
  • {@link ConcurrencyLimiterBlockingBackPressureHandler}: Limits the maximum number of messages that can be + * processed concurrently by the application.
  • + *
  • {@link ThroughputBackPressureHandler}: Adapts the throughput dynamically between high and low modes in order + * to reduce SQS pull costs when few messages are coming in.
  • + *
  • {@link CompositeBackPressureHandler}: Allows combining multiple {@link BackPressureHandler} together and + * ensures they cooperate.
  • + *
+ *

+ * Below are a few examples of how common use cases can be achieved. Keep in mind you can always create your own + * {@link BackPressureHandler} implementation and if needed combine it with the provided ones thanks to the + * {@link CompositeBackPressureHandler}. + * + *

A BackPressureHandler limiting the max concurrency with high throughput

+ * + *
{@code
+	 * containerOptionsBuilder.backPressureHandlerSupplier(() -> {
+	 * 		return ConcurrencyLimiterBlockingBackPressureHandler.builder()
+	 * 			.batchSize(batchSize)
+	 * 			.totalPermits(maxConcurrentMessages)
+	 * 			.acquireTimeout(acquireTimeout)
+	 * 			.throughputConfiguration(BackPressureMode.FIXED_HIGH_THROUGHPUT)
+	 * 			.build()
+	 * }}
+ * + *

A BackPressureHandler limiting the max concurrency with dynamic throughput

+ * + *
{@code
+	 * containerOptionsBuilder.backPressureHandlerSupplier(() -> {
+	 * 		var concurrencyLimiterBlockingBackPressureHandler = ConcurrencyLimiterBlockingBackPressureHandler.builder()
+	 * 			.batchSize(batchSize)
+	 * 			.totalPermits(maxConcurrentMessages)
+	 * 			.acquireTimeout(acquireTimeout)
+	 * 			.throughputConfiguration(BackPressureMode.AUTO)
+	 * 			.build()
+	 * 		var throughputBackPressureHandler = ThroughputBackPressureHandler.builder()
+	 * 			.batchSize(batchSize)
+	 * 			.build();
+	 * 		return new CompositeBackPressureHandler(List.of(
+	 * 				concurrencyLimiterBlockingBackPressureHandler,
+	 * 				throughputBackPressureHandler
+	 * 			),
+	 * 			batchSize,
+	 * 			standbyLimitPollingInterval
+	 * 		);
+	 * }}
* * @param backPressureHandlerSupplier the BackPressureHandler supplier. * @return this instance. diff --git a/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java new file mode 100644 index 000000000..31617c405 --- /dev/null +++ b/spring-cloud-aws-sqs/src/main/java/io/awspring/cloud/sqs/listener/SemaphoreBackPressureHandler.java @@ -0,0 +1,269 @@ +/* + * Copyright 2013-2022 the original author or authors. + * + * Licensed 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 + * + * https://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 io.awspring.cloud.sqs.listener; + +import java.time.Duration; +import java.util.Arrays; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.util.Assert; + +/** + * {@link BackPressureHandler} implementation that uses a {@link Semaphore} for handling backpressure. + * + * @author Tomaz Fernandes + * @since 3.0 + * @see io.awspring.cloud.sqs.listener.source.PollingMessageSource + */ +public class SemaphoreBackPressureHandler implements BatchAwareBackPressureHandler, IdentifiableContainerComponent { + + private static final Logger logger = LoggerFactory.getLogger(SemaphoreBackPressureHandler.class); + + private final Semaphore semaphore; + + private final int batchSize; + + private final int totalPermits; + + private final Duration acquireTimeout; + + private final BackPressureMode backPressureConfiguration; + + private volatile CurrentThroughputMode currentThroughputMode; + + private final AtomicBoolean hasAcquiredFullPermits = new AtomicBoolean(false); + + private String id; + + private SemaphoreBackPressureHandler(Builder builder) { + this.batchSize = builder.batchSize; + this.totalPermits = builder.totalPermits; + this.acquireTimeout = builder.acquireTimeout; + this.backPressureConfiguration = builder.backPressureMode; + this.semaphore = new Semaphore(totalPermits); + this.currentThroughputMode = BackPressureMode.FIXED_HIGH_THROUGHPUT.equals(backPressureConfiguration) + ? CurrentThroughputMode.HIGH + : CurrentThroughputMode.LOW; + logger.debug("SemaphoreBackPressureHandler created with configuration {} and {} total permits", + backPressureConfiguration, totalPermits); + } + + public static Builder builder() { + return new Builder(); + } + + @Override + public void setId(String id) { + this.id = id; + } + + @Override + public String getId() { + return this.id; + } + + @Override + public int request(int amount) throws InterruptedException { + return tryAcquire(amount, this.currentThroughputMode) ? amount : 0; + } + + // @formatter:off + @Override + public int requestBatch() throws InterruptedException { + return CurrentThroughputMode.LOW.equals(this.currentThroughputMode) + ? requestInLowThroughputMode() + : requestInHighThroughputMode(); + } + + private int requestInHighThroughputMode() throws InterruptedException { + return tryAcquire(this.batchSize, CurrentThroughputMode.HIGH) + ? this.batchSize + : tryAcquirePartial(); + } + // @formatter:on + + private int tryAcquirePartial() throws InterruptedException { + int availablePermits = this.semaphore.availablePermits(); + if (availablePermits == 0 || BackPressureMode.ALWAYS_POLL_MAX_MESSAGES.equals(this.backPressureConfiguration)) { + return 0; + } + int permitsToRequest = Math.min(availablePermits, this.batchSize); + CurrentThroughputMode currentThroughputModeNow = this.currentThroughputMode; + logger.trace("Trying to acquire partial batch of {} permits from {} available for {} in TM {}", + permitsToRequest, availablePermits, this.id, currentThroughputModeNow); + boolean hasAcquiredPartial = tryAcquire(permitsToRequest, currentThroughputModeNow); + return hasAcquiredPartial ? permitsToRequest : 0; + } + + private int requestInLowThroughputMode() throws InterruptedException { + // Although LTM can be set / unset by many processes, only the MessageSource thread gets here, + // so no actual concurrency + logger.debug("Trying to acquire full permits for {}. Permits left: {}", this.id, + this.semaphore.availablePermits()); + boolean hasAcquired = tryAcquire(this.totalPermits, CurrentThroughputMode.LOW); + if (hasAcquired) { + logger.debug("Acquired full permits for {}. Permits left: {}", this.id, this.semaphore.availablePermits()); + // We've acquired all permits - there's no other process currently processing messages + if (!this.hasAcquiredFullPermits.compareAndSet(false, true)) { + logger.warn("hasAcquiredFullPermits was already true. Permits left: {}", + this.semaphore.availablePermits()); + } + return this.batchSize; + } + else { + return 0; + } + } + + private boolean tryAcquire(int amount, CurrentThroughputMode currentThroughputModeNow) throws InterruptedException { + logger.trace("Acquiring {} permits for {} in TM {}", amount, this.id, this.currentThroughputMode); + boolean hasAcquired = this.semaphore.tryAcquire(amount, this.acquireTimeout.toMillis(), TimeUnit.MILLISECONDS); + if (hasAcquired) { + logger.trace("{} permits acquired for {} in TM {}. Permits left: {}", amount, this.id, + currentThroughputModeNow, this.semaphore.availablePermits()); + } + else { + logger.trace("Not able to acquire {} permits in {} milliseconds for {} in TM {}. Permits left: {}", amount, + this.acquireTimeout.toMillis(), this.id, currentThroughputModeNow, + this.semaphore.availablePermits()); + } + return hasAcquired; + } + + @Override + public void releaseBatch() { + maybeSwitchToLowThroughputMode(); + int permitsToRelease = getPermitsToRelease(this.batchSize); + this.semaphore.release(permitsToRelease); + logger.trace("Released {} permits for {}. Permits left: {}", permitsToRelease, this.id, + this.semaphore.availablePermits()); + } + + @Override + public int getBatchSize() { + return this.batchSize; + } + + private void maybeSwitchToLowThroughputMode() { + if (!BackPressureMode.FIXED_HIGH_THROUGHPUT.equals(this.backPressureConfiguration) + && CurrentThroughputMode.HIGH.equals(this.currentThroughputMode)) { + logger.debug("Entire batch of permits released for {}, setting TM LOW. Permits left: {}", this.id, + this.semaphore.availablePermits()); + this.currentThroughputMode = CurrentThroughputMode.LOW; + } + } + + @Override + public void release(int amount) { + logger.trace("Releasing {} permits for {}. Permits left: {}", amount, this.id, + this.semaphore.availablePermits()); + maybeSwitchToHighThroughputMode(amount); + int permitsToRelease = getPermitsToRelease(amount); + this.semaphore.release(permitsToRelease); + logger.trace("Released {} permits for {}. Permits left: {}", permitsToRelease, this.id, + this.semaphore.availablePermits()); + } + + @Override + public void release(int amount, ReleaseReason reason) { + if (amount == this.batchSize && reason == ReleaseReason.NONE_FETCHED) { + releaseBatch(); + } + else { + release(amount); + } + } + + private int getPermitsToRelease(int amount) { + return this.hasAcquiredFullPermits.compareAndSet(true, false) + // The first process that gets here should release all permits except for inflight messages + // We can have only one batch of messages at this point since we have all permits + ? this.totalPermits - (this.batchSize - amount) + : amount; + } + + private void maybeSwitchToHighThroughputMode(int amount) { + if (CurrentThroughputMode.LOW.equals(this.currentThroughputMode)) { + logger.debug("{} unused permit(s), setting TM HIGH for {}. Permits left: {}", amount, this.id, + this.semaphore.availablePermits()); + this.currentThroughputMode = CurrentThroughputMode.HIGH; + } + } + + @Override + public boolean drain(Duration timeout) { + logger.debug("Waiting for up to {} seconds for approx. {} permits to be released for {}", timeout.getSeconds(), + this.totalPermits - this.semaphore.availablePermits(), this.id); + try { + return this.semaphore.tryAcquire(this.totalPermits, (int) timeout.getSeconds(), TimeUnit.SECONDS); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException("Interrupted while waiting to acquire permits", e); + } + } + + private enum CurrentThroughputMode { + + HIGH, + + LOW; + + } + + public static class Builder { + + private int batchSize; + + private int totalPermits; + + private Duration acquireTimeout; + + private BackPressureMode backPressureMode; + + public Builder batchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + public Builder totalPermits(int totalPermits) { + this.totalPermits = totalPermits; + return this; + } + + public Builder acquireTimeout(Duration acquireTimeout) { + this.acquireTimeout = acquireTimeout; + return this; + } + + public Builder throughputConfiguration(BackPressureMode backPressureConfiguration) { + this.backPressureMode = backPressureConfiguration; + return this; + } + + public SemaphoreBackPressureHandler build() { + Assert.noNullElements( + Arrays.asList(this.batchSize, this.totalPermits, this.acquireTimeout, this.backPressureMode), + "Missing configuration"); + return new SemaphoreBackPressureHandler(this); + } + + } + +} diff --git a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSourceTests.java b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSourceTests.java index 0d83aca27..14e80cb07 100644 --- a/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSourceTests.java +++ b/spring-cloud-aws-sqs/src/test/java/io/awspring/cloud/sqs/listener/source/AbstractPollingMessageSourceTests.java @@ -166,6 +166,7 @@ else if (hasMadeSecondPoll.compareAndSet(false, true)) { } catch (Throwable t) { logger.error("Error (not expecting it)", t); + errors.add(t); throw new RuntimeException(t); } }, threadPool).whenComplete((v, t) -> {