From 8697e78ca88838b2b13ac40704227afe5a32f176 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 23 Sep 2025 01:22:19 +0300 Subject: [PATCH 01/75] [feat][broker/proxy] PIP-442: Add memory limits for CommandGetTopicsOfNamespace and CommandWatchTopicList --- conf/broker.conf | 24 + conf/proxy.conf | 24 + conf/standalone.conf | 24 + .../util/memory/AsyncDualMemoryLimiter.java | 61 ++ .../memory/AsyncDualMemoryLimiterPermit.java | 47 ++ .../common/util/memory/AsyncSemaphore.java | 52 ++ .../util/memory/AsyncSemaphorePermit.java | 40 + .../memory/AsyncSemaphorePermitResult.java | 47 ++ .../pulsar/broker/ServiceConfiguration.java | 42 + .../pulsar/broker/service/BrokerService.java | 13 + .../broker/service/PulsarCommandSender.java | 11 +- .../service/PulsarCommandSenderImpl.java | 33 +- .../pulsar/broker/service/ServerCnx.java | 122 ++- .../broker/service/TopicListService.java | 19 +- .../service/MessageCumulativeAckTest.java | 4 +- ...sistentDispatcherFailoverConsumerTest.java | 6 +- .../broker/service/PersistentTopicTest.java | 9 +- .../broker/service/TopicListServiceTest.java | 2 +- ...umerBackPressureMultipleConsumersTest.java | 175 +++++ .../api/PatternConsumerBackPressureTest.java | 2 + .../pulsar/common/protocol/Commands.java | 17 +- .../semaphore/AsyncDualMemoryLimiter.java | 104 +++ .../semaphore/AsyncDualMemoryLimiterImpl.java | 140 ++++ .../semaphore/AsyncDualMemoryLimiterUtil.java | 111 +++ .../common/semaphore/AsyncSemaphore.java | 94 +++ .../common/semaphore/AsyncSemaphoreImpl.java | 252 ++++++ .../pulsar/common/semaphore/package-info.java | 23 + .../AsyncDualMemoryLimiterImplTest.java | 722 ++++++++++++++++++ .../AsyncDualMemoryLimiterUtilTest.java | 497 ++++++++++++ .../semaphore/AsyncSemaphoreImplTest.java | 404 ++++++++++ .../proxy/server/LookupProxyHandler.java | 81 +- .../proxy/server/ProxyConfiguration.java | 36 + .../pulsar/proxy/server/ProxyService.java | 15 + 33 files changed, 3161 insertions(+), 92 deletions(-) create mode 100644 org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiter.java create mode 100644 org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiterPermit.java create mode 100644 org/apache/pulsar/common/util/memory/AsyncSemaphore.java create mode 100644 org/apache/pulsar/common/util/memory/AsyncSemaphorePermit.java create mode 100644 org/apache/pulsar/common/util/memory/AsyncSemaphorePermitResult.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/package-info.java create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 8fd0e18af3696..1f52bd482d5ae 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -567,6 +567,30 @@ allowOverrideEntryFilters=false # Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic maxConcurrentLookupRequest=50000 +# Maximum heap memory for inflight topic list operations (MB). +# Default: 100 MB (supports ~1M topic names assuming 100 bytes each) +maxTopicListInFlightHeapMemSizeMB=100 + +# Maximum direct memory for inflight topic list responses (MB). +# Default: 100 MB (network buffers for serialized responses) +maxTopicListInFlightDirectMemSizeMB=100 + +# Timeout for acquiring heap memory permits (milliseconds). +# Default: 25000 (25 seconds) +maxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis=25000 + +# Maximum queue size for heap memory permit requests. +# Default: 10000 (prevent unbounded queueing) +maxTopicListInFlightHeapMemSizePermitsAcquireQueueSize=10000 + +# Timeout for acquiring direct memory permits (milliseconds). +# Default: 25000 (25 seconds) +maxTopicListInFlightDirectMemSizePermitsAcquireTimeoutMillis=25000 + +# Maximum queue size for direct memory permit requests. +# Default: 10000 (prevent unbounded queueing) +maxTopicListInFlightDirectMemSizePermitsAcquireQueueSize=10000 + # Max number of concurrent topic loading request broker allows to control number of zk-operations maxConcurrentTopicLoadRequest=5000 diff --git a/conf/proxy.conf b/conf/proxy.conf index dc1fc5f002f6b..6cfe8213fab13 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -228,6 +228,30 @@ maxConcurrentInboundConnectionsPerIp=0 # Max concurrent outbound connections. The proxy will error out requests beyond that. maxConcurrentLookupRequests=50000 +# Maximum heap memory for inflight topic list operations (MB). +# Default: 100 MB (supports ~1M topic names assuming 100 bytes each) +maxTopicListInFlightHeapMemSizeMB=100 + +# Maximum direct memory for inflight topic list responses (MB). +# Default: 100 MB (network buffers for serialized responses) +maxTopicListInFlightDirectMemSizeMB=100 + +# Timeout for acquiring heap memory permits (milliseconds). +# Default: 25000 (25 seconds) +maxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis=25000 + +# Maximum queue size for heap memory permit requests. +# Default: 10000 (prevent unbounded queueing) +maxTopicListInFlightHeapMemSizePermitsAcquireQueueSize=10000 + +# Timeout for acquiring direct memory permits (milliseconds). +# Default: 25000 (25 seconds) +maxTopicListInFlightDirectMemSizePermitsAcquireTimeoutMillis=25000 + +# Maximum queue size for direct memory permit requests. +# Default: 10000 (prevent unbounded queueing) +maxTopicListInFlightDirectMemSizePermitsAcquireQueueSize=10000 + ##### --- TLS --- ##### # Deprecated - use servicePortTls and webServicePortTls instead diff --git a/conf/standalone.conf b/conf/standalone.conf index 708d4905b8ab3..f95dbd3b2a3b4 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -324,6 +324,30 @@ preciseDispatcherFlowControl=false # Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic maxConcurrentLookupRequest=50000 +# Maximum heap memory for inflight topic list operations (MB). +# Default: 100 MB (supports ~1M topic names assuming 100 bytes each) +maxTopicListInFlightHeapMemSizeMB=100 + +# Maximum direct memory for inflight topic list responses (MB). +# Default: 100 MB (network buffers for serialized responses) +maxTopicListInFlightDirectMemSizeMB=100 + +# Timeout for acquiring heap memory permits (milliseconds). +# Default: 25000 (25 seconds) +maxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis=25000 + +# Maximum queue size for heap memory permit requests. +# Default: 10000 (prevent unbounded queueing) +maxTopicListInFlightHeapMemSizePermitsAcquireQueueSize=10000 + +# Timeout for acquiring direct memory permits (milliseconds). +# Default: 25000 (25 seconds) +maxTopicListInFlightDirectMemSizePermitsAcquireTimeoutMillis=25000 + +# Maximum queue size for direct memory permit requests. +# Default: 10000 (prevent unbounded queueing) +maxTopicListInFlightDirectMemSizePermitsAcquireQueueSize=10000 + # Max number of concurrent topic loading request broker allows to control number of zk-operations maxConcurrentTopicLoadRequest=5000 diff --git a/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiter.java b/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiter.java new file mode 100644 index 0000000000000..6260dd0fdffce --- /dev/null +++ b/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiter.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.util.memory; + +import java.util.concurrent.CompletableFuture; + +/** + * Interface for asynchronous dual memory limiting. + * Manages limits for both heap and direct memory usage. + */ +public interface AsyncDualMemoryLimiter { + + /** + * Enum representing the type of memory limit. + */ + enum LimitType { + HEAP_MEMORY, // For heap memory allocation + DIRECT_MEMORY // For direct memory allocation + } + + /** + * Acquire memory permits asynchronously. + * + * @param memorySize the amount of memory to acquire in bytes + * @param limitType the type of memory limit + * @return CompletableFuture containing the memory permit + */ + CompletableFuture acquire(long memorySize, LimitType limitType); + + /** + * Update an existing permit with a new memory size. + * + * @param permit the existing permit to update + * @param newMemorySize the new memory size in bytes + * @return CompletableFuture containing the updated permit + */ + CompletableFuture update(AsyncDualMemoryLimiterPermit permit, long newMemorySize); + + /** + * Release a memory permit back to the limiter. + * + * @param permit the permit to release + */ + void release(AsyncDualMemoryLimiterPermit permit); +} diff --git a/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiterPermit.java b/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiterPermit.java new file mode 100644 index 0000000000000..043f2dcf48188 --- /dev/null +++ b/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiterPermit.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.util.memory; + +/** + * Represents a permit acquired from an AsyncDualMemoryLimiter. + * Contains information about the acquired memory allocation. + */ +public interface AsyncDualMemoryLimiterPermit { + + /** + * Get the memory size allocated by this permit. + * + * @return the memory size in bytes + */ + long getMemorySize(); + + /** + * Get the limit type associated with this permit. + * + * @return the limit type + */ + AsyncDualMemoryLimiter.LimitType getLimitType(); + + /** + * Check if this permit is still valid. + * + * @return true if the permit is valid, false otherwise + */ + boolean isValid(); +} diff --git a/org/apache/pulsar/common/util/memory/AsyncSemaphore.java b/org/apache/pulsar/common/util/memory/AsyncSemaphore.java new file mode 100644 index 0000000000000..38db6fda79727 --- /dev/null +++ b/org/apache/pulsar/common/util/memory/AsyncSemaphore.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.util.memory; + +import java.util.concurrent.CompletableFuture; + +/** + * Interface for asynchronous semaphore operations. + * Used for managing concurrent access to limited resources. + */ +public interface AsyncSemaphore { + + /** + * Acquire permits asynchronously. + * + * @param permits the number of permits to acquire + * @return CompletableFuture containing the permit result + */ + CompletableFuture acquire(long permits); + + /** + * Update an existing permit with a new permit count. + * + * @param permit the existing permit to update + * @param newPermits the new number of permits + * @return CompletableFuture containing the updated permit + */ + CompletableFuture update(AsyncSemaphorePermit permit, long newPermits); + + /** + * Release a permit back to the semaphore. + * + * @param permit the permit to release + */ + void release(AsyncSemaphorePermit permit); +} diff --git a/org/apache/pulsar/common/util/memory/AsyncSemaphorePermit.java b/org/apache/pulsar/common/util/memory/AsyncSemaphorePermit.java new file mode 100644 index 0000000000000..3c176f8bdd6e4 --- /dev/null +++ b/org/apache/pulsar/common/util/memory/AsyncSemaphorePermit.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.util.memory; + +/** + * Represents a permit acquired from an AsyncSemaphore. + * Contains information about the acquired permits. + */ +public interface AsyncSemaphorePermit { + + /** + * Get the number of permits held by this permit. + * + * @return the number of permits + */ + long getPermits(); + + /** + * Check if this permit is still valid. + * + * @return true if the permit is valid, false otherwise + */ + boolean isValid(); +} diff --git a/org/apache/pulsar/common/util/memory/AsyncSemaphorePermitResult.java b/org/apache/pulsar/common/util/memory/AsyncSemaphorePermitResult.java new file mode 100644 index 0000000000000..d254ccddad87f --- /dev/null +++ b/org/apache/pulsar/common/util/memory/AsyncSemaphorePermitResult.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.util.memory; + +/** + * Represents the result of an asynchronous semaphore acquisition. + * Contains either a permit or information about why the acquisition failed. + */ +public interface AsyncSemaphorePermitResult { + + /** + * Get the acquired permit if successful. + * + * @return the permit, or null if acquisition failed + */ + AsyncSemaphorePermit getPermit(); + + /** + * Check if the permit acquisition was successful. + * + * @return true if acquisition was successful, false otherwise + */ + boolean isSuccess(); + + /** + * Get the error message if acquisition failed. + * + * @return the error message, or null if acquisition was successful + */ + String getErrorMessage(); +} diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 5ca0db944a417..614292b18ef28 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -1386,6 +1386,48 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic") private int maxConcurrentLookupRequest = 50000; + @FieldContext( + dynamic = true, + category = CATEGORY_SERVER, + doc = "Maximum heap memory for inflight topic list operations (MB).\n" + + "Default: 100 MB (supports ~1M topic names assuming 100 bytes each)") + private int maxTopicListInFlightHeapMemSizeMB = 100; + + @FieldContext( + dynamic = true, + category = CATEGORY_SERVER, + doc = "Maximum direct memory for inflight topic list responses (MB).\n" + + "Default: 100 MB (network buffers for serialized responses)") + private int maxTopicListInFlightDirectMemSizeMB = 100; + + @FieldContext( + dynamic = true, + category = CATEGORY_SERVER, + doc = "Timeout for acquiring heap memory permits (milliseconds).\n" + + "Default: 25000 (25 seconds)") + private int maxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis = 25000; + + @FieldContext( + dynamic = true, + category = CATEGORY_SERVER, + doc = "Maximum queue size for heap memory permit requests.\n" + + "Default: 10000 (prevent unbounded queueing)") + private int maxTopicListInFlightHeapMemSizePermitsAcquireQueueSize = 10000; + + @FieldContext( + dynamic = true, + category = CATEGORY_SERVER, + doc = "Timeout for acquiring direct memory permits (milliseconds).\n" + + "Default: 25000 (25 seconds)") + private int maxTopicListInFlightDirectMemSizePermitsAcquireTimeoutMillis = 25000; + + @FieldContext( + dynamic = true, + category = CATEGORY_SERVER, + doc = "Maximum queue size for direct memory permit requests.\n" + + "Default: 10000 (prevent unbounded queueing)") + private int maxTopicListInFlightDirectMemSizePermitsAcquireQueueSize = 10000; + @FieldContext( dynamic = true, category = CATEGORY_SERVER, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 5c19de4434131..7ff1a7721e594 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -178,6 +178,7 @@ import org.apache.pulsar.common.policies.data.impl.AutoSubscriptionCreationOverrideImpl; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.protocol.schema.SchemaVersion; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FieldParser; import org.apache.pulsar.common.util.FutureUtil; @@ -249,6 +250,8 @@ public class BrokerService implements Closeable { private final SingleThreadNonConcurrentFixedRateScheduler backlogQuotaChecker; protected final AtomicReference lookupRequestSemaphore; + @Getter + private final AsyncDualMemoryLimiterImpl maxTopicListInFlightLimiter; protected final AtomicReference topicLoadRequestSemaphore; public static final String TOPIC_LOOKUP_USAGE_METRIC_NAME = "pulsar.broker.request.topic.lookup.concurrent.usage"; @@ -374,6 +377,14 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws updateConfigurationAndRegisterListeners(); this.lookupRequestSemaphore = new AtomicReference<>( new Semaphore(pulsar.getConfiguration().getMaxConcurrentLookupRequest(), false)); + // Initialize topic list memory limiter + this.maxTopicListInFlightLimiter = new AsyncDualMemoryLimiterImpl( + pulsar.getConfiguration().getMaxTopicListInFlightHeapMemSizeMB() * 1024L * 1024L, + pulsar.getConfiguration().getMaxTopicListInFlightHeapMemSizePermitsAcquireQueueSize(), + pulsar.getConfiguration().getMaxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis(), + pulsar.getConfiguration().getMaxTopicListInFlightDirectMemSizeMB() * 1024L * 1024L, + pulsar.getConfiguration().getMaxTopicListInFlightDirectMemSizePermitsAcquireQueueSize(), + pulsar.getConfiguration().getMaxTopicListInFlightDirectMemSizePermitsAcquireTimeoutMillis()); this.topicLoadRequestSemaphore = new AtomicReference<>( new Semaphore(pulsar.getConfiguration().getMaxConcurrentTopicLoadRequest(), false)); if (pulsar.getConfiguration().getMaxUnackedMessagesPerBroker() > 0 @@ -861,6 +872,8 @@ public CompletableFuture closeAsync() { } }); + maxTopicListInFlightLimiter.close(); + if (interceptor != null) { interceptor.close(); interceptor = null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java index f25703757d367..1f670ddee3bac 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java @@ -22,6 +22,7 @@ import io.netty.util.concurrent.Future; import java.util.List; import java.util.Optional; +import java.util.function.Consumer; import org.apache.bookkeeper.mledger.Entry; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse; @@ -51,8 +52,8 @@ void sendSendReceiptResponse(long producerId, long sequenceId, long highestId, l void sendSendError(long producerId, long sequenceId, ServerError error, String errorMsg); - void sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, boolean filtered, - boolean changed, long requestId); + void sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, boolean filtered, boolean changed, + long requestId, Consumer permitAcquireErrorHandler); void sendGetSchemaResponse(long requestId, SchemaInfo schema, SchemaVersion version); @@ -93,8 +94,10 @@ Future sendMessagesToConsumer(long consumerId, String topicName, Subscript void sendEndTxnErrorResponse(long requestId, TxnID txnID, ServerError error, String message); - void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics); + void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics, + Consumer permitAcquireErrorHandler); void sendWatchTopicListUpdate(long watcherId, - List newTopics, List deletedTopics, String topicsHash); + List newTopics, List deletedTopics, String topicsHash, + Consumer permitAcquireErrorHandler); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java index 105650caaaf13..730241e3118a5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandlerContext; @@ -25,6 +26,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.function.Consumer; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; import org.apache.pulsar.broker.intercept.BrokerInterceptor; @@ -37,6 +39,7 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; import org.apache.pulsar.common.util.netty.NettyChannelUtil; @Slf4j @@ -44,10 +47,13 @@ public class PulsarCommandSenderImpl implements PulsarCommandSender { private final BrokerInterceptor interceptor; private final ServerCnx cnx; + private final AsyncDualMemoryLimiter maxTopicListInFlightLimiter; - public PulsarCommandSenderImpl(BrokerInterceptor interceptor, ServerCnx cnx) { + public PulsarCommandSenderImpl(BrokerInterceptor interceptor, ServerCnx cnx, + AsyncDualMemoryLimiter maxTopicListInFlightLimiter) { this.interceptor = interceptor; this.cnx = cnx; + this.maxTopicListInFlightLimiter = maxTopicListInFlightLimiter; } @Override @@ -121,12 +127,13 @@ public void sendSendError(long producerId, long sequenceId, ServerError error, S @Override public void sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, - boolean filtered, boolean changed, long requestId) { + boolean filtered, boolean changed, long requestId, + Consumer permitAcquireErrorHandler) { BaseCommand command = Commands.newGetTopicsOfNamespaceResponseCommand(topics, topicsHash, filtered, changed, requestId); safeIntercept(command, cnx); - ByteBuf outBuf = Commands.serializeWithSize(command); - writeAndFlush(outBuf); + acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), + command, permitAcquireErrorHandler); } @Override @@ -360,9 +367,12 @@ public void sendEndTxnErrorResponse(long requestId, TxnID txnID, ServerError err * @param topics topic names which are matching, the topic name contains the partition suffix. */ @Override - public void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics) { + public void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics, + Consumer permitAcquireErrorHandler) { BaseCommand command = Commands.newWatchTopicListSuccess(requestId, watcherId, topicsHash, topics); - interceptAndWriteCommand(command); + safeIntercept(command, cnx); + acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), + command, permitAcquireErrorHandler); } /*** @@ -370,15 +380,12 @@ public void sendWatchTopicListSuccess(long requestId, long watcherId, String top */ @Override public void sendWatchTopicListUpdate(long watcherId, - List newTopics, List deletedTopics, String topicsHash) { + List newTopics, List deletedTopics, String topicsHash, + Consumer permitAcquireErrorHandler) { BaseCommand command = Commands.newWatchTopicUpdate(watcherId, newTopics, deletedTopics, topicsHash); - interceptAndWriteCommand(command); - } - - private void interceptAndWriteCommand(BaseCommand command) { safeIntercept(command, cnx); - ByteBuf outBuf = Commands.serializeWithSize(command); - writeAndFlush(outBuf); + acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), + command, permitAcquireErrorHandler); } private void writeAndFlush(ByteBuf outBuf) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index fbfb8108846b0..307134b07f7af 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -60,6 +60,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.function.BooleanSupplier; import java.util.stream.Collectors; import javax.naming.AuthenticationException; import javax.net.ssl.SSLSession; @@ -171,6 +172,8 @@ import org.apache.pulsar.common.protocol.schema.SchemaData; import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.topics.TopicsPattern; import org.apache.pulsar.common.util.FutureUtil; @@ -194,6 +197,8 @@ * parameter instance lifecycle. */ public class ServerCnx extends PulsarHandler implements TransportCnx { + // 1KB initial estimate for topic list heap permits size + private static final long INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE = 1024; private static final Logger PAUSE_RECEIVING_LOG = LoggerFactory.getLogger(ServerCnx.class.getName() + ".pauseReceiving"); private final BrokerService service; @@ -259,6 +264,7 @@ public class ServerCnx extends PulsarHandler implements TransportCnx { private final TimedSingleThreadRateLimiter requestRateLimiter; private final int pauseReceivingCooldownMilliSeconds; private boolean pausedDueToRateLimitation = false; + private AsyncDualMemoryLimiterImpl maxTopicListInFlightLimiter; // Tracks and limits number of bytes pending to be published from a single specific IO thread. static final class PendingBytesPerThreadTracker { @@ -363,6 +369,7 @@ public ServerCnx(PulsarService pulsar, String listenerName) { this.connectionController = new ConnectionController.DefaultConnectionController( conf.getBrokerMaxConnections(), conf.getBrokerMaxConnectionsPerIp()); + this.maxTopicListInFlightLimiter = pulsar.getBrokerService().getMaxTopicListInFlightLimiter(); this.enableSubscriptionPatternEvaluation = conf.isEnableBrokerSideSubscriptionPatternEvaluation(); this.maxSubscriptionPatternLength = conf.getSubscriptionPatternMaxLength(); this.topicListService = new TopicListService(pulsar, this, @@ -391,7 +398,8 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { log.debug("New connection from {}", remoteAddress); } this.ctx = ctx; - this.commandSender = new PulsarCommandSenderImpl(brokerInterceptor, this); + this.commandSender = + new PulsarCommandSenderImpl(brokerInterceptor, this, this.service.getMaxTopicListInFlightLimiter()); this.service.getPulsarStats().recordConnectionCreate(); cnxsPerThread.get().add(this); service.getPulsar().runWhenReadyForIncomingRequests(() -> { @@ -2542,45 +2550,8 @@ protected void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGet if (lookupSemaphore.tryAcquire()) { isNamespaceOperationAllowed(namespaceName, NamespaceOperation.GET_TOPICS).thenApply(isAuthorized -> { if (isAuthorized) { - getBrokerService().pulsar().getNamespaceService().getListOfUserTopics(namespaceName, mode) - .thenAccept(topics -> { - boolean filterTopics = false; - // filter system topic - List filteredTopics = topics; - - if (enableSubscriptionPatternEvaluation && topicsPattern.isPresent()) { - if (topicsPattern.get().length() <= maxSubscriptionPatternLength) { - filterTopics = true; - filteredTopics = TopicList.filterTopics(filteredTopics, topicsPattern.get(), - topicsPatternImplementation); - } else { - log.info("[{}] Subscription pattern provided [{}] was longer than maximum {}.", - remoteAddress, topicsPattern.get(), maxSubscriptionPatternLength); - } - } - String hash = TopicList.calculateHash(filteredTopics); - boolean hashUnchanged = topicsHash.isPresent() && topicsHash.get().equals(hash); - if (hashUnchanged) { - filteredTopics = Collections.emptyList(); - } - if (log.isDebugEnabled()) { - log.debug( - "[{}] Received CommandGetTopicsOfNamespace for namespace [//{}] by {}, size:{}", - remoteAddress, namespace, requestId, topics.size()); - } - commandSender.sendGetTopicsOfNamespaceResponse(filteredTopics, hash, filterTopics, - !hashUnchanged, requestId); - lookupSemaphore.release(); - }) - .exceptionally(ex -> { - log.warn("[{}] Error GetTopicsOfNamespace for namespace [//{}] by {}", - remoteAddress, namespace, requestId); - commandSender.sendErrorResponse(requestId, - BrokerServiceException.getClientErrorCode(new ServerMetadataException(ex)), - ex.getMessage()); - lookupSemaphore.release(); - return null; - }); + internalHandleGetTopicsOfNamespace(namespace, namespaceName, requestId, mode, topicsPattern, + topicsHash, lookupSemaphore); } else { final String msg = "Client is not authorized to GetTopicsOfNamespace"; log.warn("[{}] {} with role {} on namespace {}", remoteAddress, msg, getPrincipal(), namespaceName); @@ -2606,6 +2577,77 @@ protected void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGet } } + private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName namespaceName, long requestId, + CommandGetTopicsOfNamespace.Mode mode, + Optional topicsPattern, Optional topicsHash, + Semaphore lookupSemaphore) { + BooleanSupplier isPermitRequestCancelled = () -> !ctx().channel().isActive(); + maxTopicListInFlightLimiter.withAcquiredPermits(INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { + return getBrokerService().pulsar().getNamespaceService().getListOfUserTopics(namespaceName, mode) + .thenAccept(topics -> { + long actualSize = topics.stream().mapToInt(String::length).sum(); + maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, + isPermitRequestCancelled, permits -> { + boolean filterTopics = false; + // filter system topic + List filteredTopics = topics; + + if (enableSubscriptionPatternEvaluation && topicsPattern.isPresent()) { + if (topicsPattern.get().length() <= maxSubscriptionPatternLength) { + filterTopics = true; + filteredTopics = TopicList.filterTopics(filteredTopics, topicsPattern.get(), + topicsPatternImplementation); + } else { + log.info("[{}] Subscription pattern provided [{}] was longer than " + + "maximum {}.", remoteAddress, topicsPattern.get(), + maxSubscriptionPatternLength); + } + } + String hash = TopicList.calculateHash(filteredTopics); + boolean hashUnchanged = topicsHash.isPresent() && topicsHash.get().equals(hash); + if (hashUnchanged) { + filteredTopics = Collections.emptyList(); + } + if (log.isDebugEnabled()) { + log.debug("[{}] Received CommandGetTopicsOfNamespace for namespace " + + "[//{}] by {}, size:{}", remoteAddress, namespace, + requestId, + topics.size()); + } + commandSender.sendGetTopicsOfNamespaceResponse(filteredTopics, hash, filterTopics, + !hashUnchanged, requestId, ex -> { + log.warn("[{}] Failed to acquire direct memory permits for " + + "GetTopicsOfNamespace: {}", remoteAddress, ex.getMessage()); + commandSender.sendErrorResponse(requestId, ServerError.TooManyRequests, + "Cannot acquire permits for direct memory"); + }); + return CompletableFuture.completedFuture(null); + }, t -> { + log.warn("[{}] Failed to acquire heap memory permits for " + + "GetTopicsOfNamespace: {}", remoteAddress, t.getMessage()); + writeAndFlush(Commands.newError(requestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + return CompletableFuture.completedFuture(null); + }); + }).whenComplete((__, ___) -> { + lookupSemaphore.release(); + }).exceptionally(ex -> { + log.warn("[{}] Error GetTopicsOfNamespace for namespace [//{}] by {}", remoteAddress, + namespace, requestId); + commandSender.sendErrorResponse(requestId, + BrokerServiceException.getClientErrorCode(new ServerMetadataException(ex)), + ex.getMessage()); + return null; + }); + }, t -> { + log.warn("[{}] Failed to acquire initial heap memory permits for GetTopicsOfNamespace: {}", + remoteAddress, t.getMessage()); + writeAndFlush(Commands.newError(requestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + return CompletableFuture.completedFuture(null); + }); + } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index ef2ea284cf783..ad633a68ab897 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -203,7 +203,13 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo "[{}] Received WatchTopicList for namespace [//{}] by {}", connection.toString(), namespaceName, requestId); } - connection.getCommandSender().sendWatchTopicListSuccess(requestId, watcherId, hash, topicList); + connection.getCommandSender().sendWatchTopicListSuccess(requestId, watcherId, hash, topicList, + t -> { + // TODO add retry with backoff + log.warn("[{}] Cannot acquire direct memory tokens for sending topic list success." + + "State will be inconsistent on the client. {}", + connection.toString(), t.getMessage()); + }); lookupSemaphore.release(); }) .exceptionally(ex -> { @@ -288,8 +294,13 @@ public void deleteTopicListWatcher(Long watcherId) { */ public void sendTopicListUpdate(long watcherId, String topicsHash, List deletedTopics, List newTopics) { - connection.getCommandSender().sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash); + connection.getCommandSender().sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash, + t -> { + // TODO add retry with backoff + log.warn( + "[{}] Cannot acquire direct memory tokens for sending topic list update. State will be " + + "inconsistent on the client. {}", + connection.toString(), t.getMessage()); + }); } - - } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageCumulativeAckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageCumulativeAckTest.java index 4a8067b243393..2ca25c894201b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageCumulativeAckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageCumulativeAckTest.java @@ -47,6 +47,7 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; import org.apache.pulsar.common.util.Codec; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -72,7 +73,8 @@ public void setup() throws Exception { doReturn(new InetSocketAddress("localhost", 1234)).when(serverCnx).clientAddress(); when(serverCnx.getRemoteEndpointProtocolVersion()).thenReturn(ProtocolVersion.v12.getValue()); when(serverCnx.ctx()).thenReturn(mock(ChannelHandlerContext.class)); - doReturn(new PulsarCommandSenderImpl(null, serverCnx)) + AsyncDualMemoryLimiter maxTopicListInFlightLimiter = mock(AsyncDualMemoryLimiter.class); + doReturn(new PulsarCommandSenderImpl(null, serverCnx, maxTopicListInFlightLimiter)) .when(serverCnx).getCommandSender(); String topicName = TopicName.get("MessageCumulativeAckTest").toString(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java index 37cf75d84ca6d..9ac922ce46958 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java @@ -79,6 +79,7 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.apache.pulsar.common.naming.NamespaceBundle; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.awaitility.Awaitility; import org.slf4j.Logger; @@ -145,7 +146,8 @@ public void setup() throws Exception { doReturn(new InetSocketAddress("localhost", 1234)).when(serverCnx).clientAddress(); when(serverCnx.getRemoteEndpointProtocolVersion()).thenReturn(ProtocolVersion.v12.getValue()); when(serverCnx.ctx()).thenReturn(channelCtx); - doReturn(new PulsarCommandSenderImpl(null, serverCnx)) + AsyncDualMemoryLimiter maxTopicListInFlightLimiter = mock(AsyncDualMemoryLimiter.class); + doReturn(new PulsarCommandSenderImpl(null, serverCnx, maxTopicListInFlightLimiter)) .when(serverCnx).getCommandSender(); serverCnxWithOldVersion = pulsarTestContext.createServerCnxSpy(); @@ -156,7 +158,7 @@ public void setup() throws Exception { when(serverCnxWithOldVersion.getRemoteEndpointProtocolVersion()) .thenReturn(ProtocolVersion.v11.getValue()); when(serverCnxWithOldVersion.ctx()).thenReturn(channelCtx); - doReturn(new PulsarCommandSenderImpl(null, serverCnxWithOldVersion)) + doReturn(new PulsarCommandSenderImpl(null, serverCnxWithOldVersion, maxTopicListInFlightLimiter)) .when(serverCnxWithOldVersion).getCommandSender(); NamespaceService nsSvc = pulsarTestContext.getPulsarService().getNamespaceService(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index d4306a1185922..af0ad363ac339 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -134,6 +134,7 @@ import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; import org.apache.pulsar.common.protocol.ByteBufPair; import org.apache.pulsar.common.protocol.schema.SchemaVersion; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.compaction.CompactedTopic; @@ -174,6 +175,7 @@ public class PersistentTopicTest extends MockedBookKeeperTestCase { private EventLoopGroup eventLoopGroup; private ManagedLedgerFactory managedLedgerFactory; private ChannelHandlerContext ctx; + private AsyncDualMemoryLimiter maxTopicListInFlightLimiter; @BeforeMethod(alwaysRun = true) public void setup() throws Exception { @@ -209,7 +211,8 @@ public void setup() throws Exception { doReturn(true).when(serverCnx).isActive(); doReturn(true).when(serverCnx).isWritable(); doReturn(new InetSocketAddress("localhost", 1234)).when(serverCnx).clientAddress(); - doReturn(new PulsarCommandSenderImpl(null, serverCnx)) + maxTopicListInFlightLimiter = mock(AsyncDualMemoryLimiter.class); + doReturn(new PulsarCommandSenderImpl(null, serverCnx, maxTopicListInFlightLimiter)) .when(serverCnx).getCommandSender(); ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -528,7 +531,7 @@ private Producer getMockedProducerWithSpecificAddress(Topic topic, long producer doReturn(true).when(cnx).isWritable(); doReturn(new InetSocketAddress(address, 1234)).when(cnx).clientAddress(); doReturn(address.getHostAddress()).when(cnx).clientSourceAddress(); - doReturn(new PulsarCommandSenderImpl(null, cnx)).when(cnx).getCommandSender(); + doReturn(new PulsarCommandSenderImpl(null, cnx, maxTopicListInFlightLimiter)).when(cnx).getCommandSender(); return new Producer(topic, cnx, producerId, producerNameBase + producerId, role, false, null, SchemaVersion.Latest, 0, false, ProducerAccessMode.Shared, Optional.empty(), true); @@ -964,7 +967,7 @@ private Consumer getMockedConsumerWithSpecificAddress(Topic topic, Subscription doReturn(true).when(cnx).isWritable(); doReturn(new InetSocketAddress(address, 1234)).when(cnx).clientAddress(); doReturn(address.getHostAddress()).when(cnx).clientSourceAddress(); - doReturn(new PulsarCommandSenderImpl(null, cnx)).when(cnx).getCommandSender(); + doReturn(new PulsarCommandSenderImpl(null, cnx, maxTopicListInFlightLimiter)).when(cnx).getCommandSender(); return new Consumer(sub, SubType.Shared, topic.getName(), consumerId, 0, consumerNameBase + consumerId, true, cnx, role, Collections.emptyMap(), false, null, MessageId.latest, DEFAULT_CONSUMER_EPOCH); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java index 9109828c025b6..ba042470c0e3b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java @@ -91,7 +91,7 @@ public void testCommandWatchSuccessResponse() { Assert.assertEquals(1, lookupSemaphore.availablePermits()); verify(topicResources).registerPersistentTopicListener( eq(NamespaceName.get("tenant/ns")), any(TopicListService.TopicListWatcher.class)); - verify(connection.getCommandSender()).sendWatchTopicListSuccess(7, 13, hash, topics); + verify(connection.getCommandSender()).sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(topics), any()); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java new file mode 100644 index 0000000000000..da285296f8f34 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import io.netty.buffer.ByteBuf; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.stats.JvmMetrics; +import org.apache.pulsar.common.util.DirectMemoryUtils; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-impl") +public class PatternConsumerBackPressureMultipleConsumersTest extends MockedPulsarServiceBaseTest { + + @Override + @BeforeMethod + protected void setup() throws Exception { + isTcpLookup = true; + super.internalSetup(); + setupDefaultTenantAndNamespace(); + } + + @Override + @AfterMethod(alwaysRun = true) + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 60 * 1000) + public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() + throws PulsarAdminException, InterruptedException, IOException { + // number of requests to send to the broker + final int requests = 500; + // use multiple clients so that each client has a separate connection to the broker + final int numberOfClients = 200; + // create a long topic name to consume more memory per topic + final String topicName = StringUtils.repeat('a', 512) + UUID.randomUUID(); + // number of topics to create + final int topicCount = 8192; + // maximum number of requests in flight at any given time + final int maxRequestsInFlight = 200; + + // create a single topic with multiple partitions + admin.topics().createPartitionedTopic(topicName, topicCount); + + // reduce available direct memory to reproduce issues with less concurrency + long directMemoryRequired = 175 * 1024 * 1024; + List buffers = allocateDirectMemory(directMemoryRequired); + @Cleanup + Closeable releaseBuffers = () -> { + for (ByteBuf byteBuf : buffers) { + byteBuf.release(); + } + }; + + @Cleanup("shutdownNow") + final ExecutorService executorService = Executors.newFixedThreadPool(Runtime.getRuntime() + .availableProcessors()); + + @Cleanup + PulsarClientSharedResources sharedResources = + PulsarClientSharedResources.builder().build(); + List clients = new ArrayList<>(requests); + @Cleanup + Closeable closeClients = () -> { + for (PulsarClient client : clients) { + try { + client.close(); + } catch (PulsarClientException e) { + log.error("Failed to close client {}", client, e); + } + } + }; + for (int i = 0; i < numberOfClients; i++) { + PulsarClientImpl client = (PulsarClientImpl) PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .sharedResources(sharedResources) + .build(); + clients.add(client); + } + + final AtomicInteger success = new AtomicInteger(0); + final CountDownLatch latch = new CountDownLatch(requests); + final Semaphore semaphore = new Semaphore(maxRequestsInFlight); + for (int i = 0; i < requests; i++) { + PulsarClientImpl pulsarClientImpl = clients.get(i % numberOfClients); + executorService.execute(() -> { + semaphore.acquireUninterruptibly(); + try { + pulsarClientImpl.getLookup() + .getTopicsUnderNamespace(NamespaceName.get("public", "default"), + CommandGetTopicsOfNamespace.Mode.PERSISTENT, ".*", "") + .whenComplete((result, ex) -> { + semaphore.release(); + if (ex == null) { + success.incrementAndGet(); + } else { + log.error("Failed to get topic list.", ex); + } + log.info("latch-count: {}, succeed: {}, available direct mem: {} MB", latch.getCount(), + success.get(), + (DirectMemoryUtils.jvmMaxDirectMemory() - JvmMetrics.getJvmDirectMemoryUsed()) + / (1024 * 1024)); + latch.countDown(); + }); + } catch (Exception e) { + semaphore.release(); + } + }); + } + latch.await(); + Assert.assertEquals(success.get(), requests); + } + + /** + * Allocate direct memory to reduce available direct memory to the given amount of required memory. + * @param directMemoryRequired required direct memory in bytes + * @return list of ByteBufs allocated to reduce available direct memory + */ + private static List allocateDirectMemory(long directMemoryRequired) { + long usedMemory = JvmMetrics.getJvmDirectMemoryUsed(); + long maxMemory = DirectMemoryUtils.jvmMaxDirectMemory(); + long availableMemory = maxMemory - usedMemory; + List buffers = new ArrayList<>(); + if (availableMemory > directMemoryRequired) { + long allocateRemaining = availableMemory - directMemoryRequired; + log.info("Making allocations for {} MB to reduce available direct memory", + allocateRemaining / (1024 * 1024)); + int blockSize = 5 * 1024 * 1024; + while (allocateRemaining > 0) { + ByteBuf byteBuf = PulsarByteBufAllocator.DEFAULT.directBuffer(blockSize); + buffers.add(byteBuf); + allocateRemaining -= blockSize; + } + } + return buffers; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureTest.java index aa3e17c2ea03e..6f8d87c435476 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureTest.java @@ -23,6 +23,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -69,6 +70,7 @@ public void testInfiniteGetThousandsTopics() throws PulsarAdminException, Interr final int requests = 2048; final String topicName = UUID.randomUUID().toString(); admin.topics().createPartitionedTopic(topicName, topicCount); + @Cleanup("shutdownNow") final ExecutorService executorService = Executors.newFixedThreadPool(Runtime.getRuntime() .availableProcessors()); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index cab4dc8bcab0e..85e6a73f78702 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -1236,7 +1236,7 @@ public static ByteBuf newGetTopicsOfNamespaceRequest(String namespace, long requ public static BaseCommand newGetTopicsOfNamespaceResponseCommand(List topics, String topicsHash, boolean filtered, boolean changed, long requestId) { - BaseCommand cmd = localCmd(Type.GET_TOPICS_OF_NAMESPACE_RESPONSE); + BaseCommand cmd = new BaseCommand().setType(Type.GET_TOPICS_OF_NAMESPACE_RESPONSE); CommandGetTopicsOfNamespaceResponse topicsResponse = cmd.setGetTopicsOfNamespaceResponse(); topicsResponse.setRequestId(requestId); for (int i = 0; i < topics.size(); i++) { @@ -1250,12 +1250,6 @@ public static BaseCommand newGetTopicsOfNamespaceResponseCommand(List to return cmd; } - public static ByteBuf newGetTopicsOfNamespaceResponse(List topics, String topicsHash, - boolean filtered, boolean changed, long requestId) { - return serializeWithSize(newGetTopicsOfNamespaceResponseCommand( - topics, topicsHash, filtered, changed, requestId)); - } - private static final ByteBuf cmdPing; static { @@ -1635,7 +1629,7 @@ public static BaseCommand newWatchTopicList( */ public static BaseCommand newWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics) { - BaseCommand cmd = localCmd(Type.WATCH_TOPIC_LIST_SUCCESS); + BaseCommand cmd = new BaseCommand().setType(Type.WATCH_TOPIC_LIST_SUCCESS); cmd.setWatchTopicListSuccess() .setRequestId(requestId) .setWatcherId(watcherId); @@ -1654,7 +1648,7 @@ public static BaseCommand newWatchTopicListSuccess(long requestId, long watcherI */ public static BaseCommand newWatchTopicUpdate(long watcherId, List newTopics, List deletedTopics, String topicsHash) { - BaseCommand cmd = localCmd(Type.WATCH_TOPIC_UPDATE); + BaseCommand cmd = new BaseCommand().setType(Type.WATCH_TOPIC_UPDATE); cmd.setWatchTopicUpdate() .setWatcherId(watcherId) .setTopicsHash(topicsHash) @@ -1672,9 +1666,12 @@ public static BaseCommand newWatchTopicListClose(long watcherId, long requestId) } public static ByteBuf serializeWithSize(BaseCommand cmd) { + return serializeWithPrecalculatedSerializedSize(cmd, cmd.getSerializedSize()); + } + + public static ByteBuf serializeWithPrecalculatedSerializedSize(BaseCommand cmd, int cmdSize) { // / Wire format // [TOTAL_SIZE] [CMD_SIZE][CMD] - int cmdSize = cmd.getSerializedSize(); int totalSize = cmdSize + 4; int frameSize = totalSize + 4; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java new file mode 100644 index 0000000000000..53477fdbbda5c --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.semaphore; + +import java.util.concurrent.CompletableFuture; +import java.util.function.BooleanSupplier; +import java.util.function.Function; + +/** + * An abstraction for an asynchronous memory semaphore that tracks separate limits for heap and direct memory. + */ +public interface AsyncDualMemoryLimiter { + enum LimitType { + HEAP_MEMORY, // For heap memory allocation + DIRECT_MEMORY // For direct memory allocation + } + + /** + * Acquire permits for the specified memory size. + * Returned future completes when memory permits are available. + * It will complete exceptionally with AsyncDualMemoryLimiterPermitAcquireTimeoutException on timeout + * and exceptionally with AsyncDualMemoryLimiterPermitAcquireQueueFullException when queue full + * @return CompletableFuture that completes with permit when available + */ + CompletableFuture acquire(long memorySize, LimitType limitType, + BooleanSupplier isCancelled); + + /** + * Acquire or release permits for previously acquired permits by updating the requested memory size. + * Returns a future that completes when permits are available. + * It will complete exceptionally with AsyncDualMemoryLimiterPermitAcquireTimeoutException on timeout + * and exceptionally with AsyncDualMemoryLimiterPermitAcquireQueueFullException when queue full + * The provided permit is released when the permits are successfully acquired and the returned updated + * permit replaces the old instance. + * @return CompletableFuture that completes with permit when available + */ + CompletableFuture update(AsyncDualMemoryLimiterPermit permit, long newMemorySize, + BooleanSupplier isCancelled); + /** + * Release previously acquired permit. + * Must be called to prevent memory permit leaks. + */ + void release(AsyncDualMemoryLimiterPermit permit); + + /** + * Execute the specified function with acquired permits and release the permits after the returned future completes. + * @param memorySize memory size to acquire permits for + * @param limitType memory limit type to acquire permits for + * @param function function to execute with acquired permits + * @return result of the function + * @param type of the CompletableFuture returned by the function + */ + default CompletableFuture withAcquiredPermits(long memorySize, LimitType limitType, + BooleanSupplier isCancelled, + Function> function, + Function> + permitAcquireErrorHandler) { + return AsyncDualMemoryLimiterUtil.withPermitsFuture(acquire(memorySize, limitType, isCancelled), function, + permitAcquireErrorHandler, this::release); + } + + /** + * Executed the specified function with updated permits and release the permits after the returned future completes. + * @param initialPermit initial permit to update + * @param newMemorySize new memory size to update to + * @param function function to execute with updated permits + * @return result of the function + * @param type of the CompletableFuture returned by the function + */ + default CompletableFuture withUpdatedPermits(AsyncDualMemoryLimiterPermit initialPermit, long newMemorySize, + BooleanSupplier isCancelled, + Function> function, + Function> + permitAcquireErrorHandler) { + return AsyncDualMemoryLimiterUtil.withPermitsFuture(update(initialPermit, newMemorySize, isCancelled), function, + permitAcquireErrorHandler, this::release); + } + + /** + * Represents a permit for memory limiting that can be updated or released. + */ + interface AsyncDualMemoryLimiterPermit { + long getPermits(); + LimitType getLimitType(); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java new file mode 100644 index 0000000000000..33ae529e0c04e --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.semaphore; + +import io.netty.util.concurrent.DefaultThreadFactory; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.BooleanSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implementation of AsyncDualMemoryLimiter with separate limits for heap and direct memory. + */ +public class AsyncDualMemoryLimiterImpl implements AsyncDualMemoryLimiter, AutoCloseable { + private static final Logger log = LoggerFactory.getLogger(AsyncDualMemoryLimiterImpl.class); + + private final ScheduledExecutorService executor; + private final boolean shutdownExecutor; + private final AsyncSemaphoreImpl heapLimiter; + private final AsyncSemaphoreImpl directLimiter; + + public AsyncDualMemoryLimiterImpl(long maxHeapMemory, int maxHeapQueueSize, long heapTimeoutMillis, + long maxDirectMemory, int maxDirectQueueSize, long directTimeoutMillis, + ScheduledExecutorService executor) { + this(maxHeapMemory, maxHeapQueueSize, heapTimeoutMillis, maxDirectMemory, maxDirectQueueSize, + directTimeoutMillis, executor, false); + } + + public AsyncDualMemoryLimiterImpl(long maxHeapMemory, int maxHeapQueueSize, long heapTimeoutMillis, + long maxDirectMemory, int maxDirectQueueSize, long directTimeoutMillis) { + this(maxHeapMemory, maxHeapQueueSize, heapTimeoutMillis, maxDirectMemory, maxDirectQueueSize, + directTimeoutMillis, createExecutor(), true); + } + + AsyncDualMemoryLimiterImpl(long maxHeapMemory, int maxHeapQueueSize, long heapTimeoutMillis, + long maxDirectMemory, int maxDirectQueueSize, long directTimeoutMillis, + ScheduledExecutorService executor, boolean shutdownExecutor) { + this.executor = executor; + this.shutdownExecutor = shutdownExecutor; + this.heapLimiter = new AsyncSemaphoreImpl(maxHeapMemory, maxHeapQueueSize, heapTimeoutMillis, executor); + this.directLimiter = new AsyncSemaphoreImpl(maxDirectMemory, maxDirectQueueSize, directTimeoutMillis, executor); + } + + private static ScheduledExecutorService createExecutor() { + return Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("async-dual-memory-limiter")); + } + + @Override + public CompletableFuture acquire(long memorySize, LimitType limitType, + BooleanSupplier isCancelled) { + AsyncSemaphore limiter = getLimiter(limitType); + return limiter.acquire(memorySize, isCancelled).thenApply(result -> + new DualMemoryLimiterPermit(limitType, result)); + } + + private AsyncSemaphore getLimiter(LimitType limitType) { + switch (limitType) { + case HEAP_MEMORY: + return heapLimiter; + case DIRECT_MEMORY: + return directLimiter; + default: + throw new IllegalArgumentException("Unsupported limit type: " + limitType); + } + } + + @Override + public CompletableFuture update(AsyncDualMemoryLimiterPermit permit, + long newMemorySize, BooleanSupplier isCancelled) { + AsyncSemaphore limiter = getLimiter(permit.getLimitType()); + return limiter.update(castToImplementation(permit).getUnderlyingPermit(), newMemorySize, isCancelled) + .thenApply(updatedPermit -> new DualMemoryLimiterPermit(permit.getLimitType(), updatedPermit)); + } + + @Override + public void release(AsyncDualMemoryLimiterPermit permit) { + AsyncSemaphore limiter = getLimiter(permit.getLimitType()); + limiter.release(castToImplementation(permit).getUnderlyingPermit()); + } + + private DualMemoryLimiterPermit castToImplementation(AsyncDualMemoryLimiterPermit permit) { + if (permit instanceof DualMemoryLimiterPermit dualMemoryLimiterPermit) { + return dualMemoryLimiterPermit; + } else { + throw new IllegalArgumentException("Invalid permit type"); + } + } + + @Override + public void close() { + heapLimiter.close(); + directLimiter.close(); + if (shutdownExecutor) { + executor.shutdown(); + } + } + + private static class DualMemoryLimiterPermit implements AsyncDualMemoryLimiterPermit { + private final LimitType limitType; + private final AsyncSemaphore.AsyncSemaphorePermit underlyingPermit; + + DualMemoryLimiterPermit(LimitType limitType, AsyncSemaphore.AsyncSemaphorePermit underlyingPermit) { + this.limitType = limitType; + this.underlyingPermit = underlyingPermit; + } + + @Override + public long getPermits() { + return underlyingPermit.getPermits(); + } + + @Override + public LimitType getLimitType() { + return limitType; + } + + public AsyncSemaphore.AsyncSemaphorePermit getUnderlyingPermit() { + return underlyingPermit; + } + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java new file mode 100644 index 0000000000000..1b4d37ca60737 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.semaphore; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import java.util.concurrent.CompletableFuture; +import java.util.function.BooleanSupplier; +import java.util.function.Consumer; +import java.util.function.Function; +import lombok.experimental.UtilityClass; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.api.proto.BaseCommand; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter.AsyncDualMemoryLimiterPermit; + +@UtilityClass +public class AsyncDualMemoryLimiterUtil { + + public static CompletableFuture withPermitsFuture( + CompletableFuture + permitsFuture, + Function> function, + Function> + permitAcquireErrorHandler, + Consumer releaser) { + return permitsFuture + // combine the permits and error into a single pair so that it can be used in thenCompose + .handle((permits, permitAcquireError) -> + Pair.of(permits, permitAcquireError)) + .thenCompose(permitsAndError -> { + if (permitsAndError.getRight() != null) { + // permits weren't acquired + return permitAcquireErrorHandler.apply(permitsAndError.getRight()); + } else { + // permits were acquired + AsyncDualMemoryLimiterPermit permits = permitsAndError.getLeft(); + try { + return function.apply(permits) + .whenComplete((__, ___) -> + // release the permits + releaser.accept(permits)); + } catch (Throwable t) { + // release the permits if an exception occurs before the function returns + releaser.accept(permits); + throw t; + } + } + }); + } + + /** + * Acquire permits and write the command as the response to the channel. + * Releases the permits after the response has been written to the socket or the write fails. + * + * @param ctx the channel handler context. + * @param dualMemoryLimiter the memory limiter to acquire permits from. + * @param command the command to write to the channel. + * @return a future that completes when the command has been written to the channel's outbound buffer. + */ + public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush(ChannelHandlerContext ctx, + AsyncDualMemoryLimiter + dualMemoryLimiter, + BooleanSupplier isCancelled, + BaseCommand command, + Consumer + permitAcquireErrorHandler + ) { + // Calculate serialized size before acquiring permits + int serializedSize = command.getSerializedSize(); + // Acquire permits + return dualMemoryLimiter.acquire(serializedSize, AsyncDualMemoryLimiter.LimitType.DIRECT_MEMORY, isCancelled) + .whenComplete((permits, t) -> { + if (t != null) { + permitAcquireErrorHandler.accept(t); + return; + } + try { + // Serialize the response + ByteBuf outBuf = Commands.serializeWithPrecalculatedSerializedSize(command, serializedSize); + // Write the response + ctx.writeAndFlush(outBuf).addListener(future -> { + // Release permits after the response has been written to the socket + dualMemoryLimiter.release(permits); + }); + } catch (Exception e) { + // Return permits if an exception occurs before writeAndFlush is called successfully + dualMemoryLimiter.release(permits); + throw e; + } + }).thenAccept(__ -> { + }); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java new file mode 100644 index 0000000000000..3597c965839ca --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.semaphore; + +import java.util.concurrent.CompletableFuture; +import java.util.function.BooleanSupplier; + +/** + * An abstraction for a generic asynchronous semaphore. + */ +public interface AsyncSemaphore { + /** + * Acquire permits from the semaphore. + * Returned future completes when permits are available. + * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * @return CompletableFuture that completes with permit when available + */ + CompletableFuture acquire(long permits, BooleanSupplier isCancelled); + + /** + * Acquire or release permits for previously acquired permits by updating the permits. + * Returns a future that completes when permits are available. + * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * @return CompletableFuture that completes with permit when available + */ + CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, + BooleanSupplier isCancelled); + /** + * Release previously acquired permit. + * Must be called to prevent permit leaks. + */ + void release(AsyncSemaphorePermit permit); + + abstract class PermitAcquireException extends RuntimeException { + public PermitAcquireException(String message) { + super(message); + } + } + + /** + * Exception thrown when permit acquisition times out. + */ + class PermitAcquireTimeoutException extends PermitAcquireException { + public PermitAcquireTimeoutException(String message) { + super(message); + } + } + + /** + * Exception thrown when permit acquisition queue is full. + */ + class PermitAcquireQueueFullException extends PermitAcquireException { + public PermitAcquireQueueFullException(String message) { + super(message); + } + } + + class PermitAcquireAlreadyClosedException extends PermitAcquireException { + public PermitAcquireAlreadyClosedException(String message) { + super(message); + } + } + + class PermitAcquireCancelledException extends PermitAcquireException { + public PermitAcquireCancelledException(String message) { + super(message); + } + } + + /** + * Represents a permit that can be updated or released. + */ + interface AsyncSemaphorePermit { + long getPermits(); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java new file mode 100644 index 0000000000000..b1b67f5889b62 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.semaphore; + +import io.netty.util.concurrent.DefaultThreadFactory; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.function.BooleanSupplier; +import org.apache.pulsar.common.util.Runnables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implementation of AsyncSemaphore with timeout and queue size limits. + */ +public class AsyncSemaphoreImpl implements AsyncSemaphore, AutoCloseable { + private static final Logger log = LoggerFactory.getLogger(AsyncSemaphoreImpl.class); + + private final AtomicLong availablePermits; + private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + private final int maxQueueSize; + private final long timeoutMillis; + private final ScheduledExecutorService executor; + private final boolean shutdownExecutor; + private final AtomicBoolean closed = new AtomicBoolean(false); + private final Runnable processQueueRunnable = Runnables.catchingAndLoggingThrowables(this::internalProcessQueue); + + public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis) { + this(maxPermits, maxQueueSize, timeoutMillis, createExecutor(), true); + } + + public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis, + ScheduledExecutorService executor) { + this(maxPermits, maxQueueSize, timeoutMillis, executor, false); + } + + AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis, ScheduledExecutorService executor, + boolean shutdownExecutor) { + this.availablePermits = new AtomicLong(maxPermits); + this.maxQueueSize = maxQueueSize; + this.timeoutMillis = timeoutMillis; + this.executor = executor; + this.shutdownExecutor = shutdownExecutor; + } + + private static ScheduledExecutorService createExecutor() { + return Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("async-semaphore-executor")); + } + + @Override + public CompletableFuture acquire(long permits, BooleanSupplier isCancelled) { + return internalAcquire(permits, permits, isCancelled); + } + + private CompletableFuture internalAcquire(long permits, long acquirePermits, + BooleanSupplier isCancelled) { + if (permits <= 0) { + throw new IllegalArgumentException("Invalid permits value: " + permits); + } + + CompletableFuture future = new CompletableFuture<>(); + + if (closed.get()) { + future.completeExceptionally(new PermitAcquireAlreadyClosedException("Semaphore is closed")); + return future; + } + + if (queue.size() >= maxQueueSize) { + future.completeExceptionally(new PermitAcquireQueueFullException( + "Semaphore queue is full")); + return future; + } + + PendingRequest request = new PendingRequest(permits, acquirePermits, future, isCancelled); + queue.add(request); + // Schedule timeout + ScheduledFuture timeoutTask = executor.schedule(() -> { + if (!request.future.isDone() && queue.remove(request)) { + future.completeExceptionally(new PermitAcquireTimeoutException( + "Permit acquisition timed out")); + // the next request might have smaller permits and that might be processed + processQueue(); + } + }, timeoutMillis, TimeUnit.MILLISECONDS); + request.setTimeoutTask(timeoutTask); + + processQueue(); + return future; + } + + @Override + public CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, + BooleanSupplier isCancelled) { + if (newPermits <= 0) { + throw new IllegalArgumentException("Invalid permits value: " + newPermits); + } + long oldPermits = permit.getPermits(); + long additionalPermits = newPermits - oldPermits; + // mark the old permits as released without adding the permits to availablePermits + castToImplementation(permit).releasePermits(); + if (additionalPermits > 0) { + return internalAcquire(newPermits, additionalPermits, isCancelled); + } else { + // new permits are less than the old ones, so we return the difference + availablePermits.addAndGet(-additionalPermits); + processQueue(); + // return the new permits immediately + return CompletableFuture.completedFuture(new SemaphorePermit(newPermits)); + } + } + + @Override + public void release(AsyncSemaphorePermit permit) { + availablePermits.addAndGet(castToImplementation(permit).releasePermits()); + processQueue(); + } + + private SemaphorePermit castToImplementation(AsyncSemaphorePermit permit) { + if (permit instanceof SemaphorePermit semaphorePermit) { + return semaphorePermit; + } else { + throw new IllegalArgumentException("Invalid permit type"); + } + } + + private void processQueue() { + if (closed.get()) { + return; + } + executor.execute(processQueueRunnable); + } + + private void internalProcessQueue() { + while (!closed.get()) { + long current = availablePermits.get(); + if (current <= 0) { + break; + } + + PendingRequest request = queue.peek(); + if (request == null) { + break; + } + + if (request.isCancelled.getAsBoolean()) { + request.cancelTimeoutTask(); + queue.remove(request); + request.future.completeExceptionally( + new PermitAcquireCancelledException("Permit acquisition was cancelled")); + continue; + } + + if (request.acquirePermits <= current) { + availablePermits.addAndGet(-request.acquirePermits); + request.cancelTimeoutTask(); + queue.remove(request); + SemaphorePermit permit = new SemaphorePermit(request.permits); + boolean futureCompleted = request.future.complete(permit); + if (!futureCompleted){ + // request was already cancelled, return permits + availablePermits.addAndGet(request.acquirePermits); + } + } else { + break; + } + } + } + + @Override + public void close() { + if (closed.compareAndSet(false, true)) { + while (!queue.isEmpty()) { + PendingRequest request = queue.poll(); + request.cancelTimeoutTask(); + request.future.completeExceptionally(new PermitAcquireAlreadyClosedException("Semaphore is closed")); + } + if (shutdownExecutor) { + executor.shutdownNow(); + } + } + } + + private static class PendingRequest { + final long permits; + private final long acquirePermits; + final CompletableFuture future; + private final BooleanSupplier isCancelled; + private volatile ScheduledFuture timeoutTask; + + PendingRequest(long permits, long acquirePermits, CompletableFuture future, + BooleanSupplier isCancelled) { + this.permits = permits; + this.acquirePermits = acquirePermits; + this.future = future; + this.isCancelled = isCancelled; + } + + void setTimeoutTask(ScheduledFuture timeoutTask) { + this.timeoutTask = timeoutTask; + } + + void cancelTimeoutTask() { + if (timeoutTask != null) { + timeoutTask.cancel(false); + timeoutTask = null; + } + } + } + + private static class SemaphorePermit implements AsyncSemaphorePermit { + private static final AtomicLongFieldUpdater PERMITS_UPDATER = AtomicLongFieldUpdater + .newUpdater(SemaphorePermit.class, "permits"); + private volatile long permits; + + SemaphorePermit(long permits) { + this.permits = permits; + } + + @Override + public long getPermits() { + return permits; + } + + public long releasePermits() { + return PERMITS_UPDATER.getAndSet(this, 0); + } + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/package-info.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/package-info.java new file mode 100644 index 0000000000000..6d798ac9d80e3 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** + * Concurrent utilities for Pulsar. + */ +package org.apache.pulsar.common.semaphore; diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java new file mode 100644 index 0000000000000..6a079cc192e6b --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java @@ -0,0 +1,722 @@ + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.semaphore; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter.AsyncDualMemoryLimiterPermit; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter.LimitType; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireAlreadyClosedException; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireCancelledException; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireQueueFullException; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireTimeoutException; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.Test; + +public class AsyncDualMemoryLimiterImplTest { + + private AsyncDualMemoryLimiterImpl limiter; + private ScheduledExecutorService executor; + + @AfterMethod(alwaysRun = true) + public void cleanup() { + if (limiter != null) { + limiter.close(); + limiter = null; + } + if (executor != null) { + executor.shutdownNow(); + executor = null; + } + } + + @Test + public void testAcquireAndReleaseHeapMemory() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + CompletableFuture future = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + AsyncDualMemoryLimiterPermit permit = future.get(1, TimeUnit.SECONDS); + + assertNotNull(permit); + assertEquals(permit.getPermits(), 100); + assertEquals(permit.getLimitType(), LimitType.HEAP_MEMORY); + + limiter.release(permit); + } + + @Test + public void testAcquireAndReleaseDirectMemory() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + CompletableFuture future = + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false); + AsyncDualMemoryLimiterPermit permit = future.get(1, TimeUnit.SECONDS); + + assertNotNull(permit); + assertEquals(permit.getPermits(), 100); + assertEquals(permit.getLimitType(), LimitType.DIRECT_MEMORY); + + limiter.release(permit); + } + + @Test + public void testAcquireMultiplePermitsHeap() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + List permits = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + CompletableFuture future = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + permits.add(future.get(1, TimeUnit.SECONDS)); + } + + assertEquals(permits.size(), 5); + for (AsyncDualMemoryLimiterPermit permit : permits) { + assertEquals(permit.getPermits(), 100); + assertEquals(permit.getLimitType(), LimitType.HEAP_MEMORY); + limiter.release(permit); + } + } + + @Test + public void testAcquireMultiplePermitsDirect() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + List permits = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + CompletableFuture future = + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false); + permits.add(future.get(1, TimeUnit.SECONDS)); + } + + assertEquals(permits.size(), 5); + for (AsyncDualMemoryLimiterPermit permit : permits) { + assertEquals(permit.getPermits(), 100); + assertEquals(permit.getLimitType(), LimitType.DIRECT_MEMORY); + limiter.release(permit); + } + } + + @Test + public void testIndependentHeapAndDirectLimits() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + // Acquire all heap memory + AsyncDualMemoryLimiterPermit heapPermit = + limiter.acquire(1000, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + // Should still be able to acquire direct memory + AsyncDualMemoryLimiterPermit directPermit = + limiter.acquire(500, LimitType.DIRECT_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + assertNotNull(heapPermit); + assertNotNull(directPermit); + assertEquals(heapPermit.getLimitType(), LimitType.HEAP_MEMORY); + assertEquals(directPermit.getLimitType(), LimitType.DIRECT_MEMORY); + + limiter.release(heapPermit); + limiter.release(directPermit); + } + + @Test + public void testQueueingWhenHeapMemoryNotAvailable() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + // Acquire all heap memory + AsyncDualMemoryLimiterPermit permit1 = + limiter.acquire(1000, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + // Try to acquire more - should be queued + CompletableFuture future = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + assertFalse(future.isDone()); + + // Release the first permit + limiter.release(permit1); + + // The queued request should now complete + AsyncDualMemoryLimiterPermit permit2 = future.get(1, TimeUnit.SECONDS); + assertNotNull(permit2); + assertEquals(permit2.getPermits(), 100); + + limiter.release(permit2); + } + + @Test + public void testQueueingWhenDirectMemoryNotAvailable() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + // Acquire all direct memory + AsyncDualMemoryLimiterPermit permit1 = + limiter.acquire(1000, LimitType.DIRECT_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + // Try to acquire more - should be queued + CompletableFuture future = + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false); + assertFalse(future.isDone()); + + // Release the first permit + limiter.release(permit1); + + // The queued request should now complete + AsyncDualMemoryLimiterPermit permit2 = future.get(1, TimeUnit.SECONDS); + assertNotNull(permit2); + assertEquals(permit2.getPermits(), 100); + + limiter.release(permit2); + } + + @Test + public void testHeapQueueFullException() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 2, 5000, 1000, 10, 5000); + + // Acquire all heap memory + limiter.acquire(1000, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + // Fill the queue + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + + // This should fail with queue full exception + CompletableFuture future = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + + try { + future.get(1, TimeUnit.SECONDS); + fail("Expected exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireQueueFullException); + } + } + + @Test + public void testDirectQueueFullException() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 2, 5000); + + // Acquire all direct memory + limiter.acquire(1000, LimitType.DIRECT_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + // Fill the queue + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false); + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false); + + // This should fail with queue full exception + CompletableFuture future = + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false); + + try { + future.get(1, TimeUnit.SECONDS); + fail("Expected exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireQueueFullException); + } + } + + @Test + public void testHeapTimeoutException() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 100, 1000, 10, 5000); + + // Acquire all heap memory + limiter.acquire(1000, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + // Try to acquire more - should timeout + CompletableFuture future = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + + try { + future.get(2, TimeUnit.SECONDS); + fail("Expected timeout exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireTimeoutException); + } + } + + @Test + public void testDirectTimeoutException() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 100); + + // Acquire all direct memory + limiter.acquire(1000, LimitType.DIRECT_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + // Try to acquire more - should timeout + CompletableFuture future = + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false); + + try { + future.get(2, TimeUnit.SECONDS); + fail("Expected timeout exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireTimeoutException); + } + } + + @Test + public void testHeapCancellation() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 500, 1000, 10, 500); + + // Acquire all direct memory + CompletableFuture future1 = + limiter.acquire(1000, LimitType.HEAP_MEMORY, () -> false); + AsyncDualMemoryLimiterPermit permit = future1.get(1, TimeUnit.SECONDS); + + // Try to acquire more with cancellation + AtomicBoolean cancelled = new AtomicBoolean(false); + CompletableFuture future2 = + limiter.acquire(100, LimitType.HEAP_MEMORY, cancelled::get); + + assertFalse(future2.isDone()); + + // Cancel the request + cancelled.set(true); + + // Release first permit + limiter.release(permit); + + try { + future2.get(1, TimeUnit.SECONDS); + fail("Expected cancellation exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireCancelledException); + } + } + + @Test + public void testDirectCancellation() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 500, 1000, 10, 500); + + // Acquire all direct memory + CompletableFuture future1 = + limiter.acquire(1000, LimitType.DIRECT_MEMORY, () -> false); + AsyncDualMemoryLimiterPermit permit = future1.get(1, TimeUnit.SECONDS); + + // Try to acquire more with cancellation + AtomicBoolean cancelled = new AtomicBoolean(false); + CompletableFuture future2 = + limiter.acquire(100, LimitType.DIRECT_MEMORY, cancelled::get); + + assertFalse(future2.isDone()); + + // Cancel the request + cancelled.set(true); + + // Release first permit + limiter.release(permit); + + try { + future2.get(1, TimeUnit.SECONDS); + fail("Expected cancellation exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireCancelledException); + } + } + + @Test + public void testUpdateHeapPermitsIncrease() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + assertEquals(permit.getPermits(), 100); + + AsyncDualMemoryLimiterPermit updatedPermit = + limiter.update(permit, 200, () -> false).get(1, TimeUnit.SECONDS); + + assertEquals(updatedPermit.getPermits(), 200); + assertEquals(updatedPermit.getLimitType(), LimitType.HEAP_MEMORY); + + limiter.release(updatedPermit); + } + + @Test + public void testUpdateHeapPermitsDecrease() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(200, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + assertEquals(permit.getPermits(), 200); + + AsyncDualMemoryLimiterPermit updatedPermit = + limiter.update(permit, 100, () -> false).get(1, TimeUnit.SECONDS); + + assertEquals(updatedPermit.getPermits(), 100); + assertEquals(updatedPermit.getLimitType(), LimitType.HEAP_MEMORY); + + limiter.release(updatedPermit); + } + + @Test + public void testUpdateDirectPermitsIncrease() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + assertEquals(permit.getPermits(), 100); + + AsyncDualMemoryLimiterPermit updatedPermit = + limiter.update(permit, 200, () -> false).get(1, TimeUnit.SECONDS); + + assertEquals(updatedPermit.getPermits(), 200); + assertEquals(updatedPermit.getLimitType(), LimitType.DIRECT_MEMORY); + + limiter.release(updatedPermit); + } + + @Test + public void testUpdateDirectPermitsDecrease() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(200, LimitType.DIRECT_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + assertEquals(permit.getPermits(), 200); + + AsyncDualMemoryLimiterPermit updatedPermit = + limiter.update(permit, 100, () -> false).get(1, TimeUnit.SECONDS); + + assertEquals(updatedPermit.getPermits(), 100); + assertEquals(updatedPermit.getLimitType(), LimitType.DIRECT_MEMORY); + + limiter.release(updatedPermit); + } + + @Test + public void testUpdateWithInvalidPermitType() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + // Create a mock permit with invalid type + AsyncDualMemoryLimiterPermit invalidPermit = new AsyncDualMemoryLimiterPermit() { + @Override + public long getPermits() { + return 100; + } + + @Override + public LimitType getLimitType() { + return LimitType.HEAP_MEMORY; + } + }; + + try { + limiter.update(invalidPermit, 200, () -> false); + fail("Expected IllegalArgumentException"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid permit type")); + } + } + + @Test + public void testReleaseWithInvalidPermitType() { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + // Create a mock permit with invalid type + AsyncDualMemoryLimiterPermit invalidPermit = new AsyncDualMemoryLimiterPermit() { + @Override + public long getPermits() { + return 100; + } + + @Override + public LimitType getLimitType() { + return LimitType.HEAP_MEMORY; + } + }; + + try { + limiter.release(invalidPermit); + fail("Expected IllegalArgumentException"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid permit type")); + } + } + + @Test + public void testConcurrentAcquireAndReleaseHeap() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 100, 5000, 1000, 100, 5000); + + int numThreads = 10; + int numOperations = 100; + CountDownLatch startLatch = new CountDownLatch(1); + CountDownLatch doneLatch = new CountDownLatch(numThreads); + + for (int i = 0; i < numThreads; i++) { + new Thread(() -> { + try { + startLatch.await(); + for (int j = 0; j < numOperations; j++) { + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(10, LimitType.HEAP_MEMORY, () -> false) + .get(5, TimeUnit.SECONDS); + Thread.sleep(1); + limiter.release(permit); + } + } catch (Exception e) { + fail("Unexpected exception: " + e.getMessage()); + } finally { + doneLatch.countDown(); + } + }).start(); + } + + startLatch.countDown(); + assertTrue(doneLatch.await(30, TimeUnit.SECONDS)); + } + + @Test + public void testConcurrentAcquireAndReleaseDirect() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 100, 5000, 1000, 100, 5000); + + int numThreads = 10; + int numOperations = 100; + CountDownLatch startLatch = new CountDownLatch(1); + CountDownLatch doneLatch = new CountDownLatch(numThreads); + + for (int i = 0; i < numThreads; i++) { + new Thread(() -> { + try { + startLatch.await(); + for (int j = 0; j < numOperations; j++) { + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(10, LimitType.DIRECT_MEMORY, () -> false) + .get(5, TimeUnit.SECONDS); + Thread.sleep(1); + limiter.release(permit); + } + } catch (Exception e) { + fail("Unexpected exception: " + e.getMessage()); + } finally { + doneLatch.countDown(); + } + }).start(); + } + + startLatch.countDown(); + assertTrue(doneLatch.await(30, TimeUnit.SECONDS)); + } + + @Test + public void testConcurrentMixedMemoryTypes() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 100, 5000, 1000, 100, 5000); + + int numThreads = 20; // 10 for heap, 10 for direct + int numOperations = 50; + CountDownLatch startLatch = new CountDownLatch(1); + CountDownLatch doneLatch = new CountDownLatch(numThreads); + + for (int i = 0; i < numThreads; i++) { + final LimitType limitType = i < 10 + ? LimitType.HEAP_MEMORY + : LimitType.DIRECT_MEMORY; + + new Thread(() -> { + try { + startLatch.await(); + for (int j = 0; j < numOperations; j++) { + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(10, limitType, () -> false).get(5, TimeUnit.SECONDS); + Thread.sleep(1); + limiter.release(permit); + } + } catch (Exception e) { + fail("Unexpected exception: " + e.getMessage()); + } finally { + doneLatch.countDown(); + } + }).start(); + } + + startLatch.countDown(); + assertTrue(doneLatch.await(30, TimeUnit.SECONDS)); + } + + @Test + public void testCloseWithOwnExecutor() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + limiter.release(permit); + limiter.close(); + + // After close, acquire should fail + try { + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + fail("Expected exception after close"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireAlreadyClosedException); + } + } + + @Test + public void testCloseWithProvidedExecutor() throws Exception { + executor = Executors.newSingleThreadScheduledExecutor(); + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000, executor); + + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + limiter.release(permit); + + assertFalse(executor.isShutdown()); + + limiter.close(); + + // Executor should not be shut down when provided externally + assertFalse(executor.isShutdown()); + + // After close, acquire should fail + try { + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + fail("Expected exception after close"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireAlreadyClosedException); + } + } + + @Test + public void testPermitGettersHeap() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(250, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + assertEquals(permit.getPermits(), 250); + assertEquals(permit.getLimitType(), LimitType.HEAP_MEMORY); + + limiter.release(permit); + } + + @Test + public void testPermitGettersDirect() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + AsyncDualMemoryLimiterPermit permit = + limiter.acquire(350, LimitType.DIRECT_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS); + + assertEquals(permit.getPermits(), 350); + assertEquals(permit.getLimitType(), LimitType.DIRECT_MEMORY); + + limiter.release(permit); + } + + @Test + public void testMultipleReleasesProcessQueueHeap() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + // Acquire permits that fill the available memory + List permits = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + permits.add(limiter.acquire(200, LimitType.HEAP_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS)); + } + + // Queue up some requests + List> futures = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + futures.add(limiter.acquire(150, LimitType.HEAP_MEMORY, () -> false)); + } + + // All should be pending + for (CompletableFuture future : futures) { + assertFalse(future.isDone()); + } + + // Release permits one by one + for (AsyncDualMemoryLimiterPermit permit : permits) { + limiter.release(permit); + } + + // All queued requests should complete + for (CompletableFuture future : futures) { + AsyncDualMemoryLimiterPermit permit = future.get(2, TimeUnit.SECONDS); + assertNotNull(permit); + limiter.release(permit); + } + } + + @Test + public void testMultipleReleasesProcessQueueDirect() throws Exception { + limiter = new AsyncDualMemoryLimiterImpl(1000, 10, 5000, 1000, 10, 5000); + + // Acquire permits that fill the available memory + List permits = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + permits.add(limiter.acquire(200, LimitType.DIRECT_MEMORY, () -> false) + .get(1, TimeUnit.SECONDS)); + } + + // Queue up some requests + List> futures = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + futures.add(limiter.acquire(150, LimitType.DIRECT_MEMORY, () -> false)); + } + + // All should be pending + for (CompletableFuture future : futures) { + assertFalse(future.isDone()); + } + + // Release permits one by one + for (AsyncDualMemoryLimiterPermit permit : permits) { + limiter.release(permit); + } + + // All queued requests should complete + for (CompletableFuture future : futures) { + AsyncDualMemoryLimiterPermit permit = future.get(2, TimeUnit.SECONDS); + assertNotNull(permit); + limiter.release(permit); + } + } +} \ No newline at end of file diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java new file mode 100644 index 0000000000000..947812bdc2267 --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java @@ -0,0 +1,497 @@ + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.semaphore; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandlerContext; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.pulsar.common.api.proto.BaseCommand; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter.AsyncDualMemoryLimiterPermit; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter.LimitType; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireCancelledException; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class AsyncDualMemoryLimiterUtilTest { + + private AsyncDualMemoryLimiterImpl limiter; + + @BeforeMethod + public void setup() { + limiter = new AsyncDualMemoryLimiterImpl(10000, 100, 5000, 10000, 100, 5000); + } + + @AfterMethod(alwaysRun = true) + public void cleanup() { + if (limiter != null) { + limiter.close(); + limiter = null; + } + } + + @Test + public void testWithPermitsFutureSuccessfulExecution() throws Exception { + CompletableFuture permitFuture = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + + AtomicBoolean functionExecuted = new AtomicBoolean(false); + AtomicBoolean permitReleased = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.withPermitsFuture( + permitFuture, + permit -> { + functionExecuted.set(true); + return CompletableFuture.completedFuture("success"); + }, + throwable -> CompletableFuture.completedFuture("error"), + permit -> permitReleased.set(true) + ); + + String value = result.get(1, TimeUnit.SECONDS); + assertEquals(value, "success"); + assertTrue(functionExecuted.get()); + assertTrue(permitReleased.get()); + } + + @Test + public void testWithPermitsFuturePermitAcquireError() throws Exception { + CompletableFuture permitFuture = + CompletableFuture.failedFuture(new RuntimeException("Permit acquire failed")); + + AtomicBoolean functionExecuted = new AtomicBoolean(false); + AtomicBoolean errorHandlerExecuted = new AtomicBoolean(false); + AtomicBoolean permitReleased = new AtomicBoolean(false); + AtomicReference capturedError = new AtomicReference<>(); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.withPermitsFuture( + permitFuture, + permit -> { + functionExecuted.set(true); + return CompletableFuture.completedFuture("success"); + }, + throwable -> { + errorHandlerExecuted.set(true); + capturedError.set(throwable); + return CompletableFuture.completedFuture("handled-error"); + }, + permit -> permitReleased.set(true) + ); + + String value = result.get(1, TimeUnit.SECONDS); + assertEquals(value, "handled-error"); + assertTrue(errorHandlerExecuted.get()); + assertTrue(capturedError.get() instanceof RuntimeException); + assertEquals(capturedError.get().getMessage(), "Permit acquire failed"); + assertTrue(!functionExecuted.get()); // Function should not be executed + assertTrue(!permitReleased.get()); // Permits should not be released + } + + @Test + public void testWithPermitsFutureFunctionThrowsException() throws Exception { + CompletableFuture permitFuture = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + + AtomicBoolean permitReleased = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.withPermitsFuture( + permitFuture, + permit -> { + throw new RuntimeException("Function failed"); + }, + throwable -> CompletableFuture.completedFuture("error"), + permit -> permitReleased.set(true) + ); + + try { + result.get(1, TimeUnit.SECONDS); + fail("Expected exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof RuntimeException); + assertEquals(e.getCause().getMessage(), "Function failed"); + } + + // Permits should be released even when function throws + assertTrue(permitReleased.get()); + } + + @Test + public void testWithPermitsFutureFunctionReturnsFailedFuture() throws Exception { + CompletableFuture permitFuture = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + + AtomicBoolean permitReleased = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.withPermitsFuture( + permitFuture, + permit -> CompletableFuture.failedFuture(new RuntimeException("Async operation failed")), + throwable -> CompletableFuture.completedFuture("error"), + permit -> permitReleased.set(true) + ); + + try { + result.get(1, TimeUnit.SECONDS); + fail("Expected exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof RuntimeException); + assertEquals(e.getCause().getMessage(), "Async operation failed"); + } + + // Permits should be released after the returned future completes + assertTrue(permitReleased.get()); + } + + @Test + public void testWithPermitsFutureReleasesPermitsAfterAsyncCompletion() throws Exception { + CompletableFuture permitFuture = + limiter.acquire(100, LimitType.HEAP_MEMORY, () -> false); + + CompletableFuture asyncOperation = new CompletableFuture<>(); + AtomicBoolean permitReleased = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.withPermitsFuture( + permitFuture, + permit -> asyncOperation, + throwable -> CompletableFuture.completedFuture("error"), + permit -> permitReleased.set(true) + ); + + // Permits should not be released yet + Thread.sleep(100); + assertTrue(!permitReleased.get()); + assertTrue(!result.isDone()); + + // Complete the async operation + asyncOperation.complete("async-result"); + + String value = result.get(1, TimeUnit.SECONDS); + assertEquals(value, "async-result"); + assertTrue(permitReleased.get()); + } + + @Test + public void testWithPermitsFutureWithNullPermit() throws Exception { + // Simulate a null permit scenario (edge case) + CompletableFuture permitFuture = + CompletableFuture.completedFuture(null); + + AtomicBoolean functionExecuted = new AtomicBoolean(false); + AtomicBoolean permitReleased = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.withPermitsFuture( + permitFuture, + permit -> { + functionExecuted.set(true); + return CompletableFuture.completedFuture("success"); + }, + throwable -> CompletableFuture.completedFuture("error"), + permit -> permitReleased.set(true) + ); + + String value = result.get(1, TimeUnit.SECONDS); + assertEquals(value, "success"); + assertTrue(functionExecuted.get()); + assertTrue(permitReleased.get()); + } + + @Test + public void testAcquireDirectMemoryPermitsAndWriteAndFlushSuccess() throws Exception { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + ChannelFuture channelFuture = mock(ChannelFuture.class); + BaseCommand command = createTestCommand(); + + AtomicReference capturedBuf = new AtomicReference<>(); + AtomicBoolean listenerCalled = new AtomicBoolean(false); + + when(ctx.writeAndFlush(any(ByteBuf.class))).thenAnswer(invocation -> { + capturedBuf.set(invocation.getArgument(0)); + return channelFuture; + }); + + when(channelFuture.addListener(any())).thenAnswer(invocation -> { + io.netty.util.concurrent.GenericFutureListener listener = invocation.getArgument(0); + listenerCalled.set(true); + // Simulate successful write + listener.operationComplete(channelFuture); + return channelFuture; + }); + + when(channelFuture.isSuccess()).thenReturn(true); + + AtomicBoolean errorHandlerCalled = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush( + ctx, + limiter, + () -> false, + command, + throwable -> errorHandlerCalled.set(true) + ); + + result.get(1, TimeUnit.SECONDS); + + verify(ctx, times(1)).writeAndFlush(any(ByteBuf.class)); + assertTrue(listenerCalled.get()); + assertTrue(!errorHandlerCalled.get()); + assertNotNull(capturedBuf.get()); + } + + @Test + public void testAcquireDirectMemoryPermitsAndWriteAndFlushPermitAcquireError() throws Exception { + // Fill up the limiter to cause permit acquisition to fail + limiter.close(); + limiter = new AsyncDualMemoryLimiterImpl(100, 1, 100, 100, 1, 100); + + // Acquire all memory + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false).get(1, TimeUnit.SECONDS); + + // Try to acquire more - will be queued and timeout + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + BaseCommand command = createTestCommand(); + + AtomicBoolean errorHandlerCalled = new AtomicBoolean(false); + AtomicReference capturedError = new AtomicReference<>(); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush( + ctx, + limiter, + () -> false, + command, + throwable -> { + errorHandlerCalled.set(true); + capturedError.set(throwable); + } + ); + + // Wait for timeout + Thread.sleep(200); + + assertTrue(errorHandlerCalled.get()); + assertNotNull(capturedError.get()); + verify(ctx, never()).writeAndFlush(any(ByteBuf.class)); + } + + @Test + public void testAcquireDirectMemoryPermitsAndWriteAndFlushSerializationError() throws Exception { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + BaseCommand command = mock(BaseCommand.class); + + // Mock the command to throw exception during serialization + when(command.getSerializedSize()).thenReturn(100); + + AtomicBoolean errorHandlerCalled = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush( + ctx, + limiter, + () -> false, + command, + throwable -> errorHandlerCalled.set(true) + ); + + try { + result.get(1, TimeUnit.SECONDS); + fail("Expected exception"); + } catch (ExecutionException e) { + // Expected exception from serialization + assertTrue(e.getCause() != null); + } + } + + @Test + public void testAcquireDirectMemoryPermitsAndWriteAndFlushWriteException() throws Exception { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + BaseCommand command = createTestCommand(); + + // Mock writeAndFlush to throw exception + when(ctx.writeAndFlush(any(ByteBuf.class))) + .thenThrow(new RuntimeException("Write failed")); + + AtomicBoolean errorHandlerCalled = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush( + ctx, + limiter, + () -> false, + command, + throwable -> errorHandlerCalled.set(true) + ); + + try { + result.get(1, TimeUnit.SECONDS); + fail("Expected exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof RuntimeException); + assertEquals(e.getCause().getMessage(), "Write failed"); + } + } + + @Test + public void testAcquireDirectMemoryPermitsAndWriteAndFlushCancelled() throws Exception { + // Fill up the limiter so permit acquisition will be queued + limiter.close(); + limiter = new AsyncDualMemoryLimiterImpl(100, 10, 5000, 100, 10, 5000); + AsyncDualMemoryLimiterPermit permits = + limiter.acquire(100, LimitType.DIRECT_MEMORY, () -> false).get(1, TimeUnit.SECONDS); + + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + BaseCommand command = createTestCommand(); + + AtomicBoolean cancelled = new AtomicBoolean(false); + AtomicBoolean errorHandlerCalled = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush( + ctx, + limiter, + cancelled::get, + command, + throwable -> errorHandlerCalled.set(true) + ); + + // Cancel the request + cancelled.set(true); + + limiter.release(permits); + + assertFalse(errorHandlerCalled.get()); + verify(ctx, never()).writeAndFlush(any(ByteBuf.class)); + assertTrue(result.isCompletedExceptionally()); + assertThatThrownBy(() -> result.get()).hasCauseInstanceOf(PermitAcquireCancelledException.class); + } + + @Test + public void testAcquireDirectMemoryPermitsAndWriteAndFlushMultipleConcurrent() throws Exception { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + ChannelFuture channelFuture = mock(ChannelFuture.class); + + when(ctx.writeAndFlush(any(ByteBuf.class))).thenReturn(channelFuture); + when(channelFuture.addListener(any())).thenAnswer(invocation -> { + io.netty.util.concurrent.GenericFutureListener listener = invocation.getArgument(0); + listener.operationComplete(channelFuture); + return channelFuture; + }); + when(channelFuture.isSuccess()).thenReturn(true); + + int numRequests = 10; + CompletableFuture[] futures = new CompletableFuture[numRequests]; + + for (int i = 0; i < numRequests; i++) { + BaseCommand command = createTestCommand(); + futures[i] = AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush( + ctx, + limiter, + () -> false, + command, + throwable -> { + } + ); + } + + // Wait for all to complete + CompletableFuture.allOf(futures).get(5, TimeUnit.SECONDS); + + verify(ctx, times(numRequests)).writeAndFlush(any(ByteBuf.class)); + } + + @Test + public void testAcquireDirectMemoryPermitsAndWriteAndFlushReleasesOnWriteFailure() throws Exception { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + ChannelFuture channelFuture = mock(ChannelFuture.class); + BaseCommand command = createTestCommand(); + + when(ctx.writeAndFlush(any(ByteBuf.class))).thenReturn(channelFuture); + when(channelFuture.addListener(any())).thenAnswer(invocation -> { + io.netty.util.concurrent.GenericFutureListener listener = invocation.getArgument(0); + // Simulate write failure + when(channelFuture.isSuccess()).thenReturn(false); + when(channelFuture.cause()).thenReturn(new RuntimeException("Write to socket failed")); + listener.operationComplete(channelFuture); + return channelFuture; + }); + + AtomicBoolean errorHandlerCalled = new AtomicBoolean(false); + + CompletableFuture result = AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush( + ctx, + limiter, + () -> false, + command, + throwable -> errorHandlerCalled.set(true) + ); + + result.get(1, TimeUnit.SECONDS); + + verify(ctx, times(1)).writeAndFlush(any(ByteBuf.class)); + // Error handler should not be called for write failures (only for permit acquire failures) + assertTrue(!errorHandlerCalled.get()); + } + + @Test + public void testWithPermitsFutureMultipleConcurrent() throws Exception { + int numOperations = 20; + CompletableFuture[] futures = new CompletableFuture[numOperations]; + AtomicInteger releaseCount = new AtomicInteger(0); + + for (int i = 0; i < numOperations; i++) { + final int index = i; + CompletableFuture permitFuture = + limiter.acquire(50, LimitType.HEAP_MEMORY, () -> false); + + futures[i] = AsyncDualMemoryLimiterUtil.withPermitsFuture( + permitFuture, + permit -> CompletableFuture.completedFuture("result-" + index), + throwable -> CompletableFuture.completedFuture("error-" + index), + permit -> releaseCount.incrementAndGet() + ); + } + + CompletableFuture.allOf(futures).get(5, TimeUnit.SECONDS); + + assertEquals(releaseCount.get(), numOperations); + + for (int i = 0; i < numOperations; i++) { + assertEquals(futures[i].get(), "result-" + i); + } + } + + private BaseCommand createTestCommand() { + BaseCommand command = new BaseCommand().setType(BaseCommand.Type.PING); + command.setPing(); + return command; + } +} \ No newline at end of file diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java new file mode 100644 index 0000000000000..44d77a4ef955b --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java @@ -0,0 +1,404 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.semaphore; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.AsyncSemaphorePermit; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireAlreadyClosedException; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireCancelledException; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireQueueFullException; +import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireTimeoutException; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.Test; + +public class AsyncSemaphoreImplTest { + + private AsyncSemaphoreImpl semaphore; + + @AfterMethod(alwaysRun = true) + public void cleanup() { + if (semaphore != null) { + semaphore.close(); + semaphore = null; + } + } + + @Test + public void testAcquireAndReleaseSinglePermit() throws Exception { + semaphore = new AsyncSemaphoreImpl(1, 10, 5000); + + CompletableFuture future = semaphore.acquire(1, () -> false); + AsyncSemaphorePermit permit = future.get(1, TimeUnit.SECONDS); + + assertNotNull(permit); + assertEquals(permit.getPermits(), 1); + + semaphore.release(permit); + } + + @Test + public void testAcquireMultiplePermits() throws Exception { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + CompletableFuture future = semaphore.acquire(5, () -> false); + AsyncSemaphorePermit permit = future.get(1, TimeUnit.SECONDS); + + assertNotNull(permit); + assertEquals(permit.getPermits(), 5); + + semaphore.release(permit); + } + + @Test + public void testQueueingWhenNoPermitsAvailable() throws Exception { + semaphore = new AsyncSemaphoreImpl(5, 10, 5000); + + // Acquire all permits + CompletableFuture future1 = semaphore.acquire(5, () -> false); + AsyncSemaphorePermit permit1 = future1.get(1, TimeUnit.SECONDS); + + // Try to acquire more - should be queued + CompletableFuture future2 = semaphore.acquire(3, () -> false); + assertFalse(future2.isDone()); + + // Release permits + semaphore.release(permit1); + + // Now the queued request should complete + AsyncSemaphorePermit permit2 = future2.get(1, TimeUnit.SECONDS); + assertNotNull(permit2); + assertEquals(permit2.getPermits(), 3); + + semaphore.release(permit2); + } + + @Test + public void testQueueFullException() throws Exception { + semaphore = new AsyncSemaphoreImpl(1, 2, 5000); + + // Acquire the only permit + CompletableFuture future1 = semaphore.acquire(1, () -> false); + future1.get(1, TimeUnit.SECONDS); + + // Fill the queue + CompletableFuture future2 = semaphore.acquire(1, () -> false); + CompletableFuture future3 = semaphore.acquire(1, () -> false); + + // This should fail with queue full + CompletableFuture future4 = semaphore.acquire(1, () -> false); + + assertTrue(future4.isCompletedExceptionally()); + try { + future4.get(); + fail("Should have thrown exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireQueueFullException); + } + } + + @Test + public void testTimeoutException() throws Exception { + semaphore = new AsyncSemaphoreImpl(1, 10, 100); // 100ms timeout + + // Acquire the only permit + CompletableFuture future1 = semaphore.acquire(1, () -> false); + AsyncSemaphorePermit permit1 = future1.get(1, TimeUnit.SECONDS); + + // Try to acquire another permit - should timeout + CompletableFuture future2 = semaphore.acquire(1, () -> false); + + Thread.sleep(200); // Wait for timeout + + assertTrue(future2.isCompletedExceptionally()); + try { + future2.get(); + fail("Should have thrown exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireTimeoutException); + } + + semaphore.release(permit1); + } + + @Test + public void testCancellation() throws Exception { + semaphore = new AsyncSemaphoreImpl(1, 10, 5000); + + // Acquire the only permit + CompletableFuture future1 = semaphore.acquire(1, () -> false); + AsyncSemaphorePermit permit1 = future1.get(1, TimeUnit.SECONDS); + + // Try to acquire with cancellation flag + AtomicBoolean cancelled = new AtomicBoolean(false); + CompletableFuture future2 = semaphore.acquire(1, cancelled::get); + + assertFalse(future2.isDone()); + + // Cancel the request + cancelled.set(true); + + // Release the first permit + semaphore.release(permit1); + + // Give time for the queue to process + Thread.sleep(100); + + assertTrue(future2.isCompletedExceptionally()); + try { + future2.get(); + fail("Should have thrown exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireCancelledException); + } + } + + @Test + public void testInvalidPermits() { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + try { + semaphore.acquire(0, () -> false); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid permits value")); + } + + try { + semaphore.acquire(-1, () -> false); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid permits value")); + } + } + + @Test + public void testClose() throws Exception { + semaphore = new AsyncSemaphoreImpl(5, 10, 5000); + + // Acquire some permits + CompletableFuture future1 = semaphore.acquire(3, () -> false); + future1.get(1, TimeUnit.SECONDS); + + // Queue another request + CompletableFuture future2 = semaphore.acquire(5, () -> false); + assertFalse(future2.isDone()); + + // Close the semaphore + semaphore.close(); + + // Queued request should fail + assertTrue(future2.isCompletedExceptionally()); + try { + future2.get(); + fail("Should have thrown exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireAlreadyClosedException); + } + + // New acquisitions should fail + CompletableFuture future3 = semaphore.acquire(1, () -> false); + assertTrue(future3.isCompletedExceptionally()); + try { + future3.get(); + fail("Should have thrown exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof PermitAcquireAlreadyClosedException); + } + } + + @Test + public void testUpdatePermitsIncrease() throws Exception { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + // Acquire initial permits + CompletableFuture future1 = semaphore.acquire(5, () -> false); + AsyncSemaphorePermit permit1 = future1.get(1, TimeUnit.SECONDS); + assertEquals(permit1.getPermits(), 5); + + // Update to more permits + CompletableFuture future2 = semaphore.update(permit1, 8, () -> false); + AsyncSemaphorePermit permit2 = future2.get(1, TimeUnit.SECONDS); + + assertNotNull(permit2); + assertEquals(permit2.getPermits(), 8); + + semaphore.release(permit2); + } + + @Test + public void testUpdatePermitsDecrease() throws Exception { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + // Acquire initial permits + CompletableFuture future1 = semaphore.acquire(8, () -> false); + AsyncSemaphorePermit permit1 = future1.get(1, TimeUnit.SECONDS); + assertEquals(permit1.getPermits(), 8); + + // Update to fewer permits (should be immediate) + CompletableFuture future2 = semaphore.update(permit1, 5, () -> false); + AsyncSemaphorePermit permit2 = future2.get(100, TimeUnit.MILLISECONDS); + + assertNotNull(permit2); + assertEquals(permit2.getPermits(), 5); + assertTrue(future2.isDone()); + + semaphore.release(permit2); + } + + @Test + public void testUpdateWithInvalidPermits() throws Exception { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + CompletableFuture future = semaphore.acquire(5, () -> false); + AsyncSemaphorePermit permit = future.get(1, TimeUnit.SECONDS); + + try { + semaphore.update(permit, 0, () -> false); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid permits value")); + } + + try { + semaphore.update(permit, -1, () -> false); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid permits value")); + } + + semaphore.release(permit); + } + + @Test + public void testConcurrentAcquireAndRelease() throws Exception { + semaphore = new AsyncSemaphoreImpl(10, 100, 5000); + + List> futures = new ArrayList<>(); + + // Start multiple threads acquiring and releasing permits + for (int i = 0; i < 20; i++) { + CompletableFuture future = CompletableFuture.runAsync(() -> { + try { + for (int j = 0; j < 10; j++) { + CompletableFuture permitFuture = + semaphore.acquire(1, () -> false); + AsyncSemaphorePermit permit = permitFuture.get(5, TimeUnit.SECONDS); + Thread.sleep(10); // Hold permit briefly + semaphore.release(permit); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + futures.add(future); + } + + // Wait for all threads to complete + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])) + .get(30, TimeUnit.SECONDS); + } + + @Test + public void testTimeoutProcessesNextRequest() throws Exception { + semaphore = new AsyncSemaphoreImpl(5, 10, 500); + + // Acquire all permits + CompletableFuture future1 = semaphore.acquire(5, () -> false); + AsyncSemaphorePermit permit1 = future1.get(1, TimeUnit.SECONDS); + + // Request that will timeout (needs more permits than available) + CompletableFuture future2 = semaphore.acquire(10, () -> false); + + // make requests 250ms apart + Thread.sleep(250); + + // Request that can be satisfied + CompletableFuture future3 = semaphore.acquire(3, () -> false); + + // Release permits + semaphore.release(permit1); + + // The smaller request should succeed + AsyncSemaphorePermit permit3 = future3.get(1, TimeUnit.SECONDS); + assertNotNull(permit3); + assertEquals(permit3.getPermits(), 3); + + // Verify the first timed out + assertTrue(future2.isCompletedExceptionally()); + + semaphore.release(permit3); + } + + @Test + public void testMultipleReleasesProcessQueue() throws Exception { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + // Acquire all permits in small chunks + List permits = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + CompletableFuture future = semaphore.acquire(2, () -> false); + permits.add(future.get(1, TimeUnit.SECONDS)); + } + + // Queue multiple requests + CompletableFuture queued1 = semaphore.acquire(3, () -> false); + CompletableFuture queued2 = semaphore.acquire(4, () -> false); + CompletableFuture queued3 = semaphore.acquire(2, () -> false); + + assertFalse(queued1.isDone()); + assertFalse(queued2.isDone()); + assertFalse(queued3.isDone()); + + // Release permits one by one + semaphore.release(permits.get(0)); // 2 available + semaphore.release(permits.get(1)); // 4 available + + // First queued request should complete + Thread.sleep(100); + assertTrue(queued1.isDone()); + assertFalse(queued2.isDone()); + + semaphore.release(permits.get(2)); // 3 available (4 - 3 from queued1 + 2) + + Thread.sleep(100); + assertFalse(queued2.isDone()); // Still needs 4 + + semaphore.release(permits.get(3)); // 5 available + + // Second queued request should complete + Thread.sleep(100); + assertTrue(queued2.isDone()); + + semaphore.release(permits.get(4)); // 3 available (5 - 4 from queued2 + 2) + + // Third queued request should complete + Thread.sleep(100); + assertTrue(queued3.isDone()); + } +} \ No newline at end of file diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java index c38e2ba08c16c..7c59f4fbc3040 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.proxy.server; +import static org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush; import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandlerContext; import io.prometheus.client.Counter; @@ -26,26 +27,35 @@ import java.net.URI; import java.net.URISyntaxException; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Semaphore; +import java.util.function.BooleanSupplier; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.BinaryProtoLookupService; +import org.apache.pulsar.client.impl.ClientCnx; +import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.CommandGetSchema; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.api.proto.CommandLookupTopic; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse.LookupType; import org.apache.pulsar.common.api.proto.CommandPartitionedTopicMetadata; import org.apache.pulsar.common.api.proto.ServerError; +import org.apache.pulsar.common.lookup.GetTopicsResult; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion; import org.apache.pulsar.common.protocol.schema.SchemaVersion; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.util.netty.NettyChannelUtil; import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class LookupProxyHandler { + // 1KB initial estimate for topic list heap permits size + private static final long INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE = 1024; private final String throttlingErrorMessage = "Too many concurrent lookup and partitionsMetadata requests"; private final ProxyConnection proxyConnection; private final BrokerDiscoveryProvider discoveryProvider; @@ -84,6 +94,7 @@ public class LookupProxyHandler { "Counter of getTopicsOfNamespace requests rejected due to throttling") .create().register(); private final Semaphore lookupRequestSemaphore; + private final AsyncDualMemoryLimiterImpl maxTopicListInFlightLimiter; public LookupProxyHandler(ProxyService proxy, ProxyConnection proxyConnection) { this.discoveryProvider = proxy.getDiscoveryProvider(); @@ -93,6 +104,7 @@ public LookupProxyHandler(ProxyService proxy, ProxyConnection proxyConnection) { this.connectWithTLS = proxy.getConfiguration().isTlsEnabledWithBroker(); this.brokerServiceURL = this.connectWithTLS ? proxy.getConfiguration().getBrokerServiceURLTLS() : proxy.getConfiguration().getBrokerServiceURL(); + this.maxTopicListInFlightLimiter = proxy.getMaxTopicListInFlightLimiter(); } public void handleLookup(CommandLookupTopic lookup) { @@ -326,7 +338,7 @@ private void performGetTopicsOfNamespace(long clientRequestId, if (log.isDebugEnabled()) { log.debug("Getting connections to '{}' for getting TopicsOfNamespace '{}' with clientReq Id '{}'", - addr, namespaceName, clientRequestId); + addr, namespaceName, clientRequestId); } proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { // Connected to backend broker @@ -334,20 +346,8 @@ private void performGetTopicsOfNamespace(long clientRequestId, ByteBuf command; command = Commands.newGetTopicsOfNamespaceRequest(namespaceName, requestId, mode, topicsPattern, topicsHash); - clientCnx.newGetTopicsOfNamespace(command, requestId).whenComplete((r, t) -> { - if (t != null) { - log.warn("[{}] Failed to get TopicsOfNamespace {}: {}", - clientAddress, namespaceName, t.getMessage()); - writeAndFlush( - Commands.newError(clientRequestId, getServerError(t), t.getMessage())); - } else { - writeAndFlush( - Commands.newGetTopicsOfNamespaceResponse(r.getNonPartitionedOrPartitionTopics(), - r.getTopicsHash(), r.isFiltered(), - r.isChanged(), clientRequestId)); - } - }); + internalPerformGetTopicsOfNamespace(clientRequestId, namespaceName, clientCnx, command, requestId); proxyConnection.getConnectionPool().releaseConnection(clientCnx); }).exceptionally(ex -> { // Failed to connect to backend broker @@ -357,6 +357,59 @@ private void performGetTopicsOfNamespace(long clientRequestId, }); } + private void internalPerformGetTopicsOfNamespace(long clientRequestId, String namespaceName, ClientCnx clientCnx, + ByteBuf command, long requestId) { + BooleanSupplier isPermitRequestCancelled = () -> !proxyConnection.ctx().channel().isActive(); + maxTopicListInFlightLimiter.withAcquiredPermits(INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { + return clientCnx.newGetTopicsOfNamespace(command, requestId).whenComplete((r, t) -> { + if (t != null) { + log.warn("[{}] Failed to get TopicsOfNamespace {}: {}", clientAddress, namespaceName, + t.getMessage()); + writeAndFlush(Commands.newError(clientRequestId, getServerError(t), t.getMessage())); + } else { + long actualSize = + r.getNonPartitionedOrPartitionTopics().stream().mapToInt(String::length).sum(); + maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, + isPermitRequestCancelled, permits -> { + return handleWritingGetTopicsResponse(clientRequestId, r, isPermitRequestCancelled); + }, t2 -> { + log.warn("[{}] Failed to acquire actual heap memory permits for " + + "GetTopicsOfNamespace: {}", clientAddress, t2.getMessage()); + writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + + return CompletableFuture.completedFuture(null); + }); + } + }).thenApply(__ -> null); + }, t -> { + log.warn("[{}] Failed to acquire initial heap memory permits for GetTopicsOfNamespace: {}", + clientAddress, t.getMessage()); + writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + + return CompletableFuture.completedFuture(null); + }).exceptionally(ex -> { + writeAndFlush(Commands.newError(clientRequestId, getServerError(ex), ex.getMessage())); + return null; + }); + } + + private CompletableFuture handleWritingGetTopicsResponse(long clientRequestId, GetTopicsResult r, + BooleanSupplier isCancelled) { + BaseCommand responseCommand = Commands.newGetTopicsOfNamespaceResponseCommand( + r.getNonPartitionedOrPartitionTopics(), r.getTopicsHash(), r.isFiltered(), + r.isChanged(), clientRequestId); + return acquireDirectMemoryPermitsAndWriteAndFlush(proxyConnection.ctx(), maxTopicListInFlightLimiter, + isCancelled, responseCommand, t -> { + log.warn("[{}] Failed to acquire actual direct memory permits for GetTopicsOfNamespace: {}", + clientAddress, t.getMessage()); + writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + }); + } + public void handleGetSchema(CommandGetSchema commandGetSchema) { GET_SCHEMA_REQUESTS.inc(); if (log.isDebugEnabled()) { diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index 6db1b302c6613..9fd91a591cb44 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java @@ -480,6 +480,42 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private int maxConcurrentLookupRequests = 50000; + @FieldContext( + category = CATEGORY_RATE_LIMITING, + doc = "Maximum heap memory for inflight topic list operations (MB).\n" + + "Default: 100 MB (supports ~1M topic names assuming 100 bytes each)") + private int maxTopicListInFlightHeapMemSizeMB = 100; + + @FieldContext( + category = CATEGORY_RATE_LIMITING, + doc = "Maximum direct memory for inflight topic list responses (MB).\n" + + "Default: 100 MB (network buffers for serialized responses)") + private int maxTopicListInFlightDirectMemSizeMB = 100; + + @FieldContext( + category = CATEGORY_RATE_LIMITING, + doc = "Timeout for acquiring heap memory permits (milliseconds).\n" + + "Default: 25000 (25 seconds)") + private int maxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis = 25000; + + @FieldContext( + category = CATEGORY_RATE_LIMITING, + doc = "Maximum queue size for heap memory permit requests.\n" + + "Default: 10000 (prevent unbounded queueing)") + private int maxTopicListInFlightHeapMemSizePermitsAcquireQueueSize = 10000; + + @FieldContext( + category = CATEGORY_RATE_LIMITING, + doc = "Timeout for acquiring direct memory permits (milliseconds).\n" + + "Default: 25000 (25 seconds)") + private int maxTopicListInFlightDirectMemSizePermitsAcquireTimeoutMillis = 25000; + + @FieldContext( + category = CATEGORY_RATE_LIMITING, + doc = "Maximum queue size for direct memory permit requests.\n" + + "Default: 10000 (prevent unbounded queueing)") + private int maxTopicListInFlightDirectMemSizePermitsAcquireQueueSize = 10000; + @FieldContext( category = CATEGORY_CLIENT_AUTHENTICATION, doc = "The authentication plugin used by the Pulsar proxy to authenticate with Pulsar brokers" diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index 11afb68398e19..fe3d94affb4b8 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -66,6 +66,7 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.util.netty.DnsResolverUtil; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -154,6 +155,9 @@ public class ProxyService implements Closeable { @Getter private final ConnectionController connectionController; + @Getter + private final AsyncDualMemoryLimiterImpl maxTopicListInFlightLimiter; + private boolean gracefulShutdown = true; public ProxyService(ProxyConfiguration proxyConfig, @@ -212,6 +216,15 @@ public ProxyService(ProxyConfiguration proxyConfig, this.connectionController = new ConnectionController.DefaultConnectionController( proxyConfig.getMaxConcurrentInboundConnections(), proxyConfig.getMaxConcurrentInboundConnectionsPerIp()); + + // Initialize topic list memory limiter + this.maxTopicListInFlightLimiter = new AsyncDualMemoryLimiterImpl( + proxyConfig.getMaxTopicListInFlightHeapMemSizeMB() * 1024L * 1024L, + proxyConfig.getMaxTopicListInFlightHeapMemSizePermitsAcquireQueueSize(), + proxyConfig.getMaxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis(), + proxyConfig.getMaxTopicListInFlightDirectMemSizeMB() * 1024L * 1024L, + proxyConfig.getMaxTopicListInFlightDirectMemSizePermitsAcquireQueueSize(), + proxyConfig.getMaxTopicListInFlightDirectMemSizePermitsAcquireTimeoutMillis()); } public void start() throws Exception { @@ -391,6 +404,8 @@ public void close() throws IOException { dnsAddressResolverGroup.close(); + maxTopicListInFlightLimiter.close(); + if (discoveryProvider != null) { discoveryProvider.close(); } From 72e15e46926f7c714171f0a1c239ba17ead09919 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 9 Oct 2025 17:48:15 +0300 Subject: [PATCH 02/75] Allow acquiring zero permits to simplify logic when payload is empty --- .../common/semaphore/AsyncSemaphoreImpl.java | 4 +-- .../semaphore/AsyncSemaphoreImplTest.java | 29 ++++++++++--------- 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index b1b67f5889b62..18f25e37092e5 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -78,7 +78,7 @@ public CompletableFuture acquire(long permits, BooleanSupp private CompletableFuture internalAcquire(long permits, long acquirePermits, BooleanSupplier isCancelled) { - if (permits <= 0) { + if (permits < 0) { throw new IllegalArgumentException("Invalid permits value: " + permits); } @@ -115,7 +115,7 @@ private CompletableFuture internalAcquire(long permits, lo @Override public CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, BooleanSupplier isCancelled) { - if (newPermits <= 0) { + if (newPermits < 0) { throw new IllegalArgumentException("Invalid permits value: " + newPermits); } long oldPermits = permit.getPermits(); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java index 44d77a4ef955b..0331c6aba973d 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java @@ -182,13 +182,6 @@ public void testCancellation() throws Exception { public void testInvalidPermits() { semaphore = new AsyncSemaphoreImpl(10, 10, 5000); - try { - semaphore.acquire(0, () -> false); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid permits value")); - } - try { semaphore.acquire(-1, () -> false); fail("Should have thrown exception"); @@ -278,13 +271,6 @@ public void testUpdateWithInvalidPermits() throws Exception { CompletableFuture future = semaphore.acquire(5, () -> false); AsyncSemaphorePermit permit = future.get(1, TimeUnit.SECONDS); - try { - semaphore.update(permit, 0, () -> false); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid permits value")); - } - try { semaphore.update(permit, -1, () -> false); fail("Should have thrown exception"); @@ -295,6 +281,21 @@ public void testUpdateWithInvalidPermits() throws Exception { semaphore.release(permit); } + @Test + public void testUpdateWithZeroPermitsShouldReleasePermits() throws Exception { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + CompletableFuture future = semaphore.acquire(5, () -> false); + AsyncSemaphorePermit permit = future.get(1, TimeUnit.SECONDS); + + AsyncSemaphorePermit updatedPermit = semaphore.update(permit, 0, () -> false).get(1, TimeUnit.SECONDS); + + AsyncSemaphorePermit permit2 = semaphore.acquire(10, () -> false).get(1, TimeUnit.SECONDS); + + semaphore.release(updatedPermit); + semaphore.release(permit2); + } + @Test public void testConcurrentAcquireAndRelease() throws Exception { semaphore = new AsyncSemaphoreImpl(10, 100, 5000); From aad9a6f3d9828c26158586ea7a5fccba70db924c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 9 Oct 2025 17:51:57 +0300 Subject: [PATCH 03/75] Fix javadoc --- .../pulsar/common/semaphore/AsyncDualMemoryLimiter.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java index 53477fdbbda5c..cdb3c85ebb2a4 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java @@ -34,8 +34,8 @@ enum LimitType { /** * Acquire permits for the specified memory size. * Returned future completes when memory permits are available. - * It will complete exceptionally with AsyncDualMemoryLimiterPermitAcquireTimeoutException on timeout - * and exceptionally with AsyncDualMemoryLimiterPermitAcquireQueueFullException when queue full + * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full * @return CompletableFuture that completes with permit when available */ CompletableFuture acquire(long memorySize, LimitType limitType, @@ -44,8 +44,8 @@ CompletableFuture acquire(long memorySize, LimitTy /** * Acquire or release permits for previously acquired permits by updating the requested memory size. * Returns a future that completes when permits are available. - * It will complete exceptionally with AsyncDualMemoryLimiterPermitAcquireTimeoutException on timeout - * and exceptionally with AsyncDualMemoryLimiterPermitAcquireQueueFullException when queue full + * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full * The provided permit is released when the permits are successfully acquired and the returned updated * permit replaces the old instance. * @return CompletableFuture that completes with permit when available From fa4b3a721c7399a31efae02c661f2a3f1a4ef4f6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 13:09:48 +0300 Subject: [PATCH 04/75] Add test for the proxy lookups --- ...umerBackPressureMultipleConsumersTest.java | 12 ++- ...umerBackPressureMultipleConsumersTest.java | 87 +++++++++++++++++++ 2 files changed, 97 insertions(+), 2 deletions(-) create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java index da285296f8f34..1e086c0b57129 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java @@ -81,7 +81,7 @@ public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() admin.topics().createPartitionedTopic(topicName, topicCount); // reduce available direct memory to reproduce issues with less concurrency - long directMemoryRequired = 175 * 1024 * 1024; + long directMemoryRequired = getDirectMemoryRequiredMB() * 1024 * 1024; List buffers = allocateDirectMemory(directMemoryRequired); @Cleanup Closeable releaseBuffers = () -> { @@ -110,7 +110,7 @@ public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() }; for (int i = 0; i < numberOfClients; i++) { PulsarClientImpl client = (PulsarClientImpl) PulsarClient.builder() - .serviceUrl(pulsar.getBrokerServiceUrl()) + .serviceUrl(getClientServiceUrl()) .sharedResources(sharedResources) .build(); clients.add(client); @@ -149,6 +149,14 @@ public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() Assert.assertEquals(success.get(), requests); } + protected int getDirectMemoryRequiredMB() { + return 175; + } + + protected String getClientServiceUrl() { + return pulsar.getBrokerServiceUrl(); + } + /** * Allocate direct memory to reduce available direct memory to the given amount of required memory. * @param directMemoryRequired required direct memory in bytes diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java new file mode 100644 index 0000000000000..f7245f3b9e911 --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.proxy.server; + +import static org.mockito.Mockito.doReturn; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.api.PatternConsumerBackPressureMultipleConsumersTest; +import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.mockito.Mockito; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; + +@Slf4j +public class ProxyPatternConsumerBackPressureMultipleConsumersTest extends + PatternConsumerBackPressureMultipleConsumersTest { + + protected ProxyService proxyService; + protected ProxyConfiguration proxyConfig = new ProxyConfiguration(); + protected Authentication proxyClientAuthentication; + + @Override + @BeforeMethod + protected void setup() throws Exception { + super.setup(); + initializeProxyConfig(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); + doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); + doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) + .createConfigurationMetadataStore(); + proxyService.start(); + } + + protected void initializeProxyConfig() throws Exception { + proxyConfig.setServicePort(Optional.ofNullable(0)); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); + proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); + + proxyClientAuthentication = AuthenticationFactory.create( + proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + } + + @Override + @AfterMethod(alwaysRun = true) + protected void cleanup() throws Exception { + proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } + super.cleanup(); + } + + @Override + protected String getClientServiceUrl() { + return proxyService.getServiceUrl(); + } + + @Override + protected int getDirectMemoryRequiredMB() { + return 225; + } +} From 8ad197fc58031028fd05a699a760ce73ff706628 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 16:05:38 +0300 Subject: [PATCH 05/75] Implement metrics --- .../TopicListMemoryLimiter.java | 273 ++++++++++++++++++ .../broker/topiclistlimit/package-info.java | 22 ++ .../prometheus/PrometheusMetricsClient.java | 2 + .../pulsar/broker/service/BrokerService.java | 5 +- ...umerBackPressureMultipleConsumersTest.java | 26 +- .../semaphore/AsyncDualMemoryLimiterImpl.java | 24 +- .../common/semaphore/AsyncSemaphore.java | 15 + .../common/semaphore/AsyncSemaphoreImpl.java | 44 ++- pulsar-proxy/pom.xml | 12 + .../pulsar/proxy/server/ProxyService.java | 10 +- ...umerBackPressureMultipleConsumersTest.java | 5 + 11 files changed, 425 insertions(+), 13 deletions(-) create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/package-info.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java new file mode 100644 index 0000000000000..2a3fadbd0f291 --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.topiclistlimit; + +import io.opentelemetry.api.metrics.DoubleGauge; +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.LongGauge; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.api.metrics.ObservableDoubleGauge; +import io.opentelemetry.api.metrics.ObservableLongUpDownCounter; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.Counter; +import io.prometheus.client.Gauge; +import io.prometheus.client.Summary; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; +import org.apache.pulsar.common.semaphore.AsyncSemaphore; + +/** + * Topic list memory limiter that exposes both Prometheus metrics and OpenTelemetry metrics. + */ +public class TopicListMemoryLimiter extends AsyncDualMemoryLimiterImpl { + private final CollectorRegistry collectorRegistry; + private final Gauge heapMemoryUsedBytes; + private final Gauge heapMemoryLimitBytes; + private final Gauge directMemoryUsedBytes; + private final Gauge directMemoryLimitBytes; + private final Gauge heapQueueSize; + private final Gauge heapQueueMaxSize; + private final Gauge directQueueSize; + private final Gauge directQueueMaxSize; + private final Summary heapWaitTimeMs; + private final Summary directWaitTimeMs; + private final Counter heapTimeoutTotal; + private final Counter directTimeoutTotal; + private final ObservableDoubleGauge otelHeapMemoryUsedGauge; + private final DoubleGauge otelHeapMemoryLimitGauge; + private final ObservableDoubleGauge otelDirectMemoryUsedGauge; + private final DoubleGauge otelDirectMemoryLimitGauge; + private final ObservableLongUpDownCounter otelHeapQueueSize; + private final ObservableLongUpDownCounter otelDirectQueueSize; + private final DoubleHistogram otelHeapWaitTime; + private final DoubleHistogram otelDirectWaitTime; + private final LongCounter otelHeapTimeoutTotal; + private final LongCounter otelDirectTimeoutTotal; + + public TopicListMemoryLimiter(CollectorRegistry collectorRegistry, String prometheusPrefix, + Meter openTelemetryMeter, + long maxHeapMemory, int maxHeapQueueSize, + long heapTimeoutMillis, long maxDirectMemory, int maxDirectQueueSize, + long directTimeoutMillis) { + super(maxHeapMemory, maxHeapQueueSize, heapTimeoutMillis, maxDirectMemory, maxDirectQueueSize, + directTimeoutMillis); + this.collectorRegistry = collectorRegistry; + + AsyncSemaphore heapMemoryLimiter = getLimiter(LimitType.HEAP_MEMORY); + AsyncSemaphore directMemoryLimiter = getLimiter(LimitType.DIRECT_MEMORY); + + this.heapMemoryUsedBytes = Gauge.build(prometheusPrefix + "topic_list_heap_memory_used_bytes", + "Current heap memory used by topic listings") + .create() + .setChild(new Gauge.Child() { + @Override + public double get() { + return heapMemoryLimiter.getAcquiredPermits(); + } + }) + .register(collectorRegistry); + this.otelHeapMemoryUsedGauge = openTelemetryMeter.gaugeBuilder("topic.list.heap.memory.used") + .setUnit("By") + .setDescription("Current heap memory used by topic listings") + .buildWithCallback(observableDoubleMeasurement -> { + observableDoubleMeasurement.record(heapMemoryLimiter.getAcquiredPermits()); + }); + + this.heapMemoryLimitBytes = Gauge.build(prometheusPrefix + "topic_list_heap_memory_limit_bytes", + "Configured heap memory limit") + .create().register(collectorRegistry); + this.heapMemoryLimitBytes.set(maxHeapMemory); + this.otelHeapMemoryLimitGauge = openTelemetryMeter.gaugeBuilder("topic.list.heap.memory.limit") + .setUnit("By") + .setDescription("Configured heap memory limit") + .build(); + this.otelHeapMemoryLimitGauge.set(maxHeapMemory); + + this.directMemoryUsedBytes = Gauge.build(prometheusPrefix + "topic_list_direct_memory_used_bytes", + "Current direct memory used by topic listings") + .create() + .setChild(new Gauge.Child() { + @Override + public double get() { + return directMemoryLimiter.getAcquiredPermits(); + } + }) + .register(collectorRegistry); + this.otelDirectMemoryUsedGauge = openTelemetryMeter.gaugeBuilder("topic.list.direct.memory.used") + .setUnit("By") + .setDescription("Current direct memory used by topic listings") + .buildWithCallback(observableDoubleMeasurement -> { + observableDoubleMeasurement.record(directMemoryLimiter.getAcquiredPermits()); + }); + + this.directMemoryLimitBytes = Gauge.build(prometheusPrefix + "topic_list_direct_memory_limit_bytes", + "Configured direct memory limit") + .create().register(collectorRegistry); + this.directMemoryLimitBytes.set(maxDirectMemory); + this.otelDirectMemoryLimitGauge = openTelemetryMeter.gaugeBuilder("topic.list.direct.memory.limit") + .setUnit("By") + .setDescription("Configured direct memory limit") + .build(); + this.otelDirectMemoryLimitGauge.set(maxHeapMemory); + + this.heapQueueSize = Gauge.build(prometheusPrefix + "topic_list_heap_queue_size", + "Current heap memory limiter queue size") + .create() + .setChild(new Gauge.Child() { + @Override + public double get() { + return heapMemoryLimiter.getQueueSize(); + } + }) + .register(collectorRegistry); + this.otelHeapQueueSize = openTelemetryMeter + .upDownCounterBuilder("topic.list.heap.queue.size") + .setDescription("Current heap memory limiter queue size") + .setUnit("1") + .buildWithCallback(observableLongMeasurement -> { + observableLongMeasurement.record(heapMemoryLimiter.getQueueSize()); + }); + + this.heapQueueMaxSize = Gauge.build(prometheusPrefix + "topic_list_heap_queue_max_size", + "Maximum heap memory limiter queue size") + .create().register(collectorRegistry); + this.heapQueueMaxSize.set(maxHeapQueueSize); + LongGauge otelHeapQueueMaxSize = openTelemetryMeter + .gaugeBuilder("topic.list.heap.queue.max.size") + .setDescription("Maximum heap memory limiter queue size") + .setUnit("1") + .ofLongs() + .build(); + otelHeapQueueMaxSize.set(maxHeapQueueSize); + + this.directQueueSize = Gauge.build(prometheusPrefix + "topic_list_direct_queue_size", + "Current direct memory limiter queue size") + .create() + .setChild(new Gauge.Child() { + @Override + public double get() { + return directMemoryLimiter.getQueueSize(); + } + }) + .register(collectorRegistry); + this.otelDirectQueueSize = openTelemetryMeter + .upDownCounterBuilder("topic.list.direct.queue.size") + .setDescription("Current direct memory limiter queue size") + .setUnit("1") + .buildWithCallback(observableLongMeasurement -> { + observableLongMeasurement.record(directMemoryLimiter.getQueueSize()); + }); + + this.directQueueMaxSize = Gauge.build(prometheusPrefix + "topic_list_direct_queue_max_size", + "Maximum direct memory limiter queue size") + .create().register(collectorRegistry); + this.directQueueMaxSize.set(maxDirectQueueSize); + LongGauge otelDirectQueueMaxSize = openTelemetryMeter + .gaugeBuilder("topic.list.direct.queue.max.size") + .setDescription("Maximum direct memory limiter queue size") + .setUnit("1") + .ofLongs() + .build(); + otelDirectQueueMaxSize.set(maxDirectQueueSize); + + this.heapWaitTimeMs = Summary.build(prometheusPrefix + "topic_list_heap_wait_time_ms", + "Wait time for heap memory permits") + .quantile(0.50, 0.01) + .quantile(0.95, 0.01) + .quantile(0.99, 0.01) + .quantile(1, 0.01) + .create().register(collectorRegistry); + this.otelHeapWaitTime = openTelemetryMeter.histogramBuilder("topic.list.heap.wait.time.ms") + .setUnit("s") + .setDescription("Wait time for heap memory permits") + .build(); + + this.directWaitTimeMs = Summary.build(prometheusPrefix + "topic_list_direct_wait_time_ms", + "Wait time for direct memory permits") + .quantile(0.50, 0.01) + .quantile(0.95, 0.01) + .quantile(0.99, 0.01) + .quantile(1, 0.01) + .create().register(collectorRegistry); + this.otelDirectWaitTime = openTelemetryMeter.histogramBuilder("topic.list.direct.wait.time.ms") + .setUnit("s") + .setDescription("Wait time for direct memory permits") + .build(); + + this.heapTimeoutTotal = Counter.build(prometheusPrefix + "topic_list_heap_timeout_total", + "Total heap memory permit timeouts") + .create().register(collectorRegistry); + this.otelHeapTimeoutTotal = openTelemetryMeter.counterBuilder("topic.list.heap.timeout.total") + .setDescription("Total heap memory permit timeouts") + .setUnit("1") + .build(); + + this.directTimeoutTotal = Counter.build(prometheusPrefix + "topic_list_direct_timeout_total", + "Total direct memory permit timeouts") + .create().register(collectorRegistry); + this.otelDirectTimeoutTotal = openTelemetryMeter.counterBuilder("topic.list.direct.timeout.total") + .setDescription("Total direct memory permit timeouts") + .setUnit("1") + .build(); + } + + @Override + protected void recordHeapWaitTime(long waitTimeNanos) { + if (waitTimeNanos == Long.MAX_VALUE) { + heapTimeoutTotal.inc(); + otelHeapTimeoutTotal.add(1); + } else { + heapWaitTimeMs.observe(TimeUnit.NANOSECONDS.toMillis(waitTimeNanos)); + otelHeapWaitTime.record(waitTimeNanos / 1_000_000_000.0d); + } + } + + @Override + protected void recordDirectWaitTime(long waitTimeNanos) { + if (waitTimeNanos == Long.MAX_VALUE) { + directTimeoutTotal.inc(); + otelDirectTimeoutTotal.add(1); + } else { + directWaitTimeMs.observe(TimeUnit.NANOSECONDS.toMillis(waitTimeNanos)); + otelDirectWaitTime.record(waitTimeNanos / 1_000_000_000.0d); + } + } + + @Override + public void close() { + super.close(); + collectorRegistry.unregister(heapMemoryUsedBytes); + collectorRegistry.unregister(heapMemoryLimitBytes); + collectorRegistry.unregister(directMemoryUsedBytes); + collectorRegistry.unregister(directMemoryLimitBytes); + collectorRegistry.unregister(heapQueueSize); + collectorRegistry.unregister(heapQueueMaxSize); + collectorRegistry.unregister(directQueueSize); + collectorRegistry.unregister(directQueueMaxSize); + collectorRegistry.unregister(heapWaitTimeMs); + collectorRegistry.unregister(directWaitTimeMs); + collectorRegistry.unregister(heapTimeoutTotal); + collectorRegistry.unregister(directTimeoutTotal); + otelHeapMemoryUsedGauge.close(); + otelDirectMemoryUsedGauge.close(); + otelHeapQueueSize.close(); + otelDirectQueueSize.close(); + } +} diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/package-info.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/package-info.java new file mode 100644 index 0000000000000..42911acf708bd --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +/** + * Topic Listing memory limiter, PIP-442 + */ +package org.apache.pulsar.broker.topiclistlimit; diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java index cf6d63e96ce84..298cbc2f65862 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java @@ -31,6 +31,7 @@ import java.util.TreeMap; import java.util.regex.Matcher; import java.util.regex.Pattern; +import lombok.Getter; import org.apache.commons.lang3.tuple.Pair; public class PrometheusMetricsClient { @@ -110,6 +111,7 @@ public boolean contains(String labelName, String labelValue) { } public static class Metrics { + @Getter final Multimap nameToDataPoints; public Metrics(Multimap nameToDataPoints) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 7ff1a7721e594..e173f8d64a239 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -42,6 +42,7 @@ import io.netty.util.concurrent.DefaultThreadFactory; import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.ObservableLongUpDownCounter; +import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; import java.io.Closeable; @@ -138,6 +139,7 @@ import org.apache.pulsar.broker.stats.prometheus.metrics.Summary; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; +import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; import org.apache.pulsar.broker.validator.BindAddressValidator; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -378,7 +380,8 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.lookupRequestSemaphore = new AtomicReference<>( new Semaphore(pulsar.getConfiguration().getMaxConcurrentLookupRequest(), false)); // Initialize topic list memory limiter - this.maxTopicListInFlightLimiter = new AsyncDualMemoryLimiterImpl( + this.maxTopicListInFlightLimiter = new TopicListMemoryLimiter( + CollectorRegistry.defaultRegistry, "pulsar_broker_", pulsar.getOpenTelemetry().getMeter(), pulsar.getConfiguration().getMaxTopicListInFlightHeapMemSizeMB() * 1024L * 1024L, pulsar.getConfiguration().getMaxTopicListInFlightHeapMemSizePermitsAcquireQueueSize(), pulsar.getConfiguration().getMaxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis(), diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java index 1e086c0b57129..fb2e302b15966 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import static org.testng.Assert.assertEquals; import io.netty.buffer.ByteBuf; import java.io.Closeable; import java.io.IOException; @@ -32,7 +33,9 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -40,7 +43,6 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.stats.JvmMetrics; import org.apache.pulsar.common.util.DirectMemoryUtils; -import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -146,7 +148,27 @@ public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() }); } latch.await(); - Assert.assertEquals(success.get(), requests); + assertEquals(success.get(), requests); + + validateTopiclistPrometheusMetrics(); + } + + private void validateTopiclistPrometheusMetrics() { + MutableInt metricsCount = new MutableInt(0); + PrometheusMetricsClient metricsClient = + new PrometheusMetricsClient("localhost", pulsar.getListenPortHTTP().get()); + PrometheusMetricsClient.Metrics metrics = metricsClient.getMetrics(); + metrics.getNameToDataPoints().entries().forEach(entry -> { + if (entry.getKey().startsWith(getMetricsPrefix())) { + metricsCount.increment(); + System.out.println(entry.getKey() + " -> " + entry.getValue()); + } + }); + assertEquals(metricsCount.intValue(), 26); + } + + protected String getMetricsPrefix() { + return "pulsar_broker_topic_list_"; } protected int getDirectMemoryRequiredMB() { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java index 33ae529e0c04e..1da47033ebc34 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java @@ -55,8 +55,10 @@ public AsyncDualMemoryLimiterImpl(long maxHeapMemory, int maxHeapQueueSize, long ScheduledExecutorService executor, boolean shutdownExecutor) { this.executor = executor; this.shutdownExecutor = shutdownExecutor; - this.heapLimiter = new AsyncSemaphoreImpl(maxHeapMemory, maxHeapQueueSize, heapTimeoutMillis, executor); - this.directLimiter = new AsyncSemaphoreImpl(maxDirectMemory, maxDirectQueueSize, directTimeoutMillis, executor); + this.heapLimiter = new AsyncSemaphoreImpl(maxHeapMemory, maxHeapQueueSize, heapTimeoutMillis, executor, + this::recordHeapWaitTime); + this.directLimiter = new AsyncSemaphoreImpl(maxDirectMemory, maxDirectQueueSize, directTimeoutMillis, executor, + this::recordDirectWaitTime); } private static ScheduledExecutorService createExecutor() { @@ -72,7 +74,7 @@ public CompletableFuture acquire(long memorySize, new DualMemoryLimiterPermit(limitType, result)); } - private AsyncSemaphore getLimiter(LimitType limitType) { + protected AsyncSemaphore getLimiter(LimitType limitType) { switch (limitType) { case HEAP_MEMORY: return heapLimiter; @@ -105,6 +107,22 @@ private DualMemoryLimiterPermit castToImplementation(AsyncDualMemoryLimiterPermi } } + /** + * Record the wait time for a heap memory allocation permit. + * @param waitTimeNanos wait time in nanoseconds, or Long.MAX_VALUE if the allocation timed out + */ + protected void recordHeapWaitTime(long waitTimeNanos) { + + } + + /** + * Record the wait time for a direct memory allocation permit. + * @param waitTimeNanos wait time in nanoseconds, or Long.MAX_VALUE if the allocation timed out + */ + protected void recordDirectWaitTime(long waitTimeNanos) { + + } + @Override public void close() { heapLimiter.close(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java index 3597c965839ca..9a8cd86c57923 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java @@ -49,6 +49,21 @@ CompletableFuture update(AsyncSemaphorePermit permit, long */ void release(AsyncSemaphorePermit permit); + /** + * Get the number of available permits. + */ + long getAvailablePermits(); + + /** + * Get the number of acquired permits. + */ + long getAcquiredPermits(); + + /** + * Get the current size of queued requests. + */ + int getQueueSize(); + abstract class PermitAcquireException extends RuntimeException { public PermitAcquireException(String message) { super(message); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index 18f25e37092e5..b3ae4fd6b081b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.function.BooleanSupplier; +import java.util.function.LongConsumer; import org.apache.pulsar.common.util.Runnables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,29 +42,33 @@ public class AsyncSemaphoreImpl implements AsyncSemaphore, AutoCloseable { private final AtomicLong availablePermits; private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + private final long maxPermits; private final int maxQueueSize; private final long timeoutMillis; private final ScheduledExecutorService executor; private final boolean shutdownExecutor; + private final LongConsumer queueLatencyRecorder; private final AtomicBoolean closed = new AtomicBoolean(false); private final Runnable processQueueRunnable = Runnables.catchingAndLoggingThrowables(this::internalProcessQueue); public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis) { - this(maxPermits, maxQueueSize, timeoutMillis, createExecutor(), true); + this(maxPermits, maxQueueSize, timeoutMillis, createExecutor(), true, null); } public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis, - ScheduledExecutorService executor) { - this(maxPermits, maxQueueSize, timeoutMillis, executor, false); + ScheduledExecutorService executor, LongConsumer queueLatencyRecorder) { + this(maxPermits, maxQueueSize, timeoutMillis, executor, false, queueLatencyRecorder); } AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis, ScheduledExecutorService executor, - boolean shutdownExecutor) { + boolean shutdownExecutor, LongConsumer queueLatencyRecorder) { this.availablePermits = new AtomicLong(maxPermits); + this.maxPermits = maxPermits; this.maxQueueSize = maxQueueSize; this.timeoutMillis = timeoutMillis; this.executor = executor; this.shutdownExecutor = shutdownExecutor; + this.queueLatencyRecorder = queueLatencyRecorder; } private static ScheduledExecutorService createExecutor() { @@ -100,6 +105,10 @@ private CompletableFuture internalAcquire(long permits, lo // Schedule timeout ScheduledFuture timeoutTask = executor.schedule(() -> { if (!request.future.isDone() && queue.remove(request)) { + // timeout is recorded with Long.MAX_VALUE as the age + recordQueueLatency(Long.MAX_VALUE); + // also record the time in the queue + recordQueueLatency(request.getAgeNanos()); future.completeExceptionally(new PermitAcquireTimeoutException( "Permit acquisition timed out")); // the next request might have smaller permits and that might be processed @@ -112,6 +121,12 @@ private CompletableFuture internalAcquire(long permits, lo return future; } + private void recordQueueLatency(long ageNanos) { + if (queueLatencyRecorder != null) { + queueLatencyRecorder.accept(ageNanos); + } + } + @Override public CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, BooleanSupplier isCancelled) { @@ -139,6 +154,21 @@ public void release(AsyncSemaphorePermit permit) { processQueue(); } + @Override + public long getAvailablePermits() { + return availablePermits.get(); + } + + @Override + public long getAcquiredPermits() { + return maxPermits - availablePermits.get(); + } + + @Override + public int getQueueSize() { + return queue.size(); + } + private SemaphorePermit castToImplementation(AsyncSemaphorePermit permit) { if (permit instanceof SemaphorePermit semaphorePermit) { return semaphorePermit; @@ -179,6 +209,7 @@ private void internalProcessQueue() { request.cancelTimeoutTask(); queue.remove(request); SemaphorePermit permit = new SemaphorePermit(request.permits); + recordQueueLatency(request.getAgeNanos()); boolean futureCompleted = request.future.complete(permit); if (!futureCompleted){ // request was already cancelled, return permits @@ -210,6 +241,7 @@ private static class PendingRequest { final CompletableFuture future; private final BooleanSupplier isCancelled; private volatile ScheduledFuture timeoutTask; + private final long requestCreatedNanos = System.nanoTime(); PendingRequest(long permits, long acquirePermits, CompletableFuture future, BooleanSupplier isCancelled) { @@ -229,6 +261,10 @@ void cancelTimeoutTask() { timeoutTask = null; } } + + long getAgeNanos() { + return System.nanoTime() - requestCreatedNanos; + } } private static class SemaphorePermit implements AsyncSemaphorePermit { diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 9531879ea8edb..42d16e22d487b 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -215,6 +215,18 @@ ${consolecaptor.version} test + + io.rest-assured + rest-assured + test + + + ${project.groupId} + pulsar-broker-common + ${project.version} + test-jar + test + diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index fe3d94affb4b8..d8aac78a97e01 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -35,6 +35,7 @@ import io.netty.resolver.dns.DnsNameResolverBuilder; import io.netty.util.concurrent.DefaultThreadFactory; import io.netty.util.concurrent.Future; +import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import java.io.Closeable; @@ -62,6 +63,7 @@ import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; +import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlets; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -150,7 +152,7 @@ public class ProxyService implements Closeable { private PrometheusMetricsServlet metricsServlet; private List pendingMetricsProviders; @Getter - private PulsarProxyOpenTelemetry openTelemetry; + private final PulsarProxyOpenTelemetry openTelemetry; @Getter private final ConnectionController connectionController; @@ -217,8 +219,11 @@ public ProxyService(ProxyConfiguration proxyConfig, proxyConfig.getMaxConcurrentInboundConnections(), proxyConfig.getMaxConcurrentInboundConnectionsPerIp()); + this.openTelemetry = new PulsarProxyOpenTelemetry(proxyConfig); + // Initialize topic list memory limiter - this.maxTopicListInFlightLimiter = new AsyncDualMemoryLimiterImpl( + this.maxTopicListInFlightLimiter = new TopicListMemoryLimiter( + CollectorRegistry.defaultRegistry, "pulsar_proxy_", openTelemetry.getMeter(), proxyConfig.getMaxTopicListInFlightHeapMemSizeMB() * 1024L * 1024L, proxyConfig.getMaxTopicListInFlightHeapMemSizePermitsAcquireQueueSize(), proxyConfig.getMaxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis(), @@ -300,7 +305,6 @@ public void start() throws Exception { } createMetricsServlet(); - openTelemetry = new PulsarProxyOpenTelemetry(proxyConfig); // Initialize the message protocol handlers. // start the protocol handlers only after the broker is ready, diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java index f7245f3b9e911..abf43a568de0a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java @@ -84,4 +84,9 @@ protected String getClientServiceUrl() { protected int getDirectMemoryRequiredMB() { return 225; } + + @Override + protected String getMetricsPrefix() { + return "pulsar_proxy_topic_list_"; + } } From 98a45e8eb0bbe9024645d288765e8ad52f41c268 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 16:52:50 +0300 Subject: [PATCH 06/75] Update PIP-442 based on implementation --- pip/pip-442.md | 578 +++++++++++++++++++++++++++++++------------------ 1 file changed, 370 insertions(+), 208 deletions(-) diff --git a/pip/pip-442.md b/pip/pip-442.md index 19e0500d2de3f..d629572f61133 100644 --- a/pip/pip-442.md +++ b/pip/pip-442.md @@ -14,8 +14,8 @@ Pulsar brokers already implement comprehensive memory management for most operat - `maxMessagePublishBufferSizeInMB` (default: 50% direct memory): Limits memory used for buffering messages during publishing, providing backpressure when producers exceed broker capacity **Managed Ledger Memory Limits:** -- `managedLedgerMaxReadsInFlightSizeInMB` (default: 0, disabled): Controls memory allocation for concurrent read operations from BookKeeper, preventing excessive memory usage during high read loads. This limit extends to cover buffers that were read from BookKeeper and are waiting in channel outbound buffers to be written to client sockets. -- `managedLedgerCacheSizeMB` (default: 20% of direct memory): Limits cache memory for recently read ledger entries, ensuring predictable memory usage for read caching. This limit extends to cover buffers that were read from the cache and are waiting in channel outbound buffers to be written to client sockets. +- `managedLedgerMaxReadsInFlightSizeInMB` (default: 0, disabled): Controls memory allocation for concurrent read operations from BookKeeper, preventing excessive memory usage during high read loads. This limit extends to cover buffers that were read from BookKeeper and are waiting in channel outbound buffers to be written to client sockets. +- `managedLedgerCacheSizeMB` (default: 20% of direct memory): Limits cache memory for recently read ledger entries, ensuring predictable memory usage for read caching. This limit extends to cover buffers that were read from the cache and are waiting in channel outbound buffers to be written to client sockets. **Additional Memory Controls:** - `maxConcurrentLookupRequest` (default: 50000): Limits concurrent topic lookup requests. The unit of this limit is the number of requests; it is not expressed in memory size. @@ -46,14 +46,14 @@ These operations can allocate arbitrary amounts of memory based on namespace siz The `getTopicsOfNamespace` request follows this path: 1. **Client Request**: Sends `CommandGetTopicsOfNamespace` via binary protocol -2. **Request Handling**: - - Broker: `ServerCnx.handleGetTopicsOfNamespace()` - - Proxy: `ProxyConnection.handleGetTopicsOfNamespace()` +2. **Request Handling**: + - Broker: `ServerCnx.handleGetTopicsOfNamespace()` + - Proxy: `LookupProxyHandler.handleGetTopicsOfNamespace()` 3. **Topic Retrieval**: `NamespaceService.getListOfUserTopics()` orchestrates: - - Fetches persistent topics from `TopicResources` - - Retrieves non-persistent topics from local cache or peer clusters - - Filters system topics using `TopicList.filterSystemTopic()` - - Implements caching via `inProgressQueryUserTopics` to prevent duplicate queries + - Fetches persistent topics from `TopicResources` + - Retrieves non-persistent topics from local cache or peer clusters + - Filters system topics using `TopicList.filterSystemTopic()` + - Implements caching via `inProgressQueryUserTopics` to prevent duplicate queries 4. **Response Construction**: Packages results with hash calculation and filtering metadata 5. **Response Transmission**: Sends complete response back to client @@ -107,49 +107,72 @@ The lack of memory limits for topic listing commands creates the final significa The solution introduces an `AsyncDualMemoryLimiter` that acts as a memory-aware semaphore for topic listing operations, completing Pulsar's memory management framework: -1. **Memory Tracking**: Before processing requests or sending responses, the system estimates memory requirements and acquires permits from the limiter. When the permit cannot be estimated and allocated before the operation, an initial permit is acquired and updated before continuing with handling. Although not optimal, this will effectively limit memory usage across the broker. +1. **Memory Tracking**: Before processing requests or sending responses, the system estimates memory requirements and acquires permits from the limiter. An initial permit is acquired with a fixed estimate (1KB), then updated to reflect the actual memory usage after the topic list is retrieved. 2. **Dual Memory Pools**: Separate tracking for heap memory (topic list assembly) and direct memory (network buffers) with independent limits, since topic listing operations use both types of memory. -3. **Asynchronous Backpressure**: When memory limits are reached, requests queue with configurable timeouts rather than failing immediately, providing graceful degradation similar to `managedLedgerMaxReadsInFlightSizeInMB` behavior. This type of solution is helpful since rejecting requests and requiring clients to retry can cause more load on the system and would cause unfair queueing. When the queue is completely full, requests are rejected. +3. **Asynchronous Backpressure**: When memory limits are reached, requests queue with configurable timeouts rather than failing immediately, providing graceful degradation similar to `managedLedgerMaxReadsInFlightSizeInMB` behavior. When the queue is completely full, requests are rejected. 4. **Graceful Degradation**: The system continues processing within memory limits rather than crashing, with clear metrics indicating when memory-based throttling occurs. 5. **Release Guarantees**: Memory permits are released after response transmission completes or on request failure, preventing memory leaks and ensuring accurate memory tracking. +6. **Cancellation Support**: The implementation supports cancellation of permit requests when the client connection is closed, preventing unnecessary queueing and resource allocation. + ## Detailed Design ### Design & Implementation Details #### AsyncSemaphore Interface -This is an abstraction for a generic asynchronous semaphore. The memory limiter implementation will use this abstraction to implement separate limiters for heap and direct memory. +This is an abstraction for a generic asynchronous semaphore. The memory limiter implementation uses this abstraction to implement separate limiters for heap and direct memory. ```java public interface AsyncSemaphore { /** * Acquire permits from the semaphore. * Returned future completes when permits are available. - * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout - * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * It will complete exceptionally with PermitAcquireTimeoutException on timeout, + * PermitAcquireQueueFullException when queue full, + * PermitAcquireCancelledException when cancelled, + * and PermitAcquireAlreadyClosedException when closed. + * @param permits number of permits to acquire + * @param isCancelled supplier to check if request should be cancelled * @return CompletableFuture that completes with permit when available */ - CompletableFuture acquire(long permits); + CompletableFuture acquire(long permits, BooleanSupplier isCancelled); /** * Acquire or release permits for previously acquired permits by updating the permits. * Returns a future that completes when permits are available. - * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout - * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * @param permit existing permit to update + * @param newPermits new permit count + * @param isCancelled supplier to check if request should be cancelled * @return CompletableFuture that completes with permit when available */ - CompletableFuture update(AsyncSemaphorePermit permit, long newPermits); + CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, + BooleanSupplier isCancelled); /** * Release previously acquired permit. * Must be called to prevent memory permit leaks. */ void release(AsyncSemaphorePermit permit); + + /** + * Get the number of available permits. + */ + long getAvailablePermits(); + + /** + * Get the number of acquired permits. + */ + long getAcquiredPermits(); + + /** + * Get the current size of queued requests. + */ + int getQueueSize(); } ``` @@ -167,223 +190,368 @@ public interface AsyncDualMemoryLimiter { /** * Acquire permits for the specified memory size. * Returned future completes when memory permits are available. - * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout - * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * @param memorySize size in bytes + * @param limitType type of memory limit + * @param isCancelled supplier to check if request should be cancelled * @return CompletableFuture that completes with permit when available */ - CompletableFuture acquire(long memorySize, LimitType limitType); + CompletableFuture acquire(long memorySize, LimitType limitType, + BooleanSupplier isCancelled); /** * Acquire or release permits for previously acquired permits by updating the requested memory size. - * Returns a future that completes when permits are available. - * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout - * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * @param permit existing permit + * @param newMemorySize new size in bytes + * @param isCancelled supplier to check if request should be cancelled * @return CompletableFuture that completes with permit when available */ - CompletableFuture update(AsyncDualMemoryLimiterPermit permit, long newMemorySize); + CompletableFuture update(AsyncDualMemoryLimiterPermit permit, + long newMemorySize, + BooleanSupplier isCancelled); /** * Release previously acquired permit. - * Must be called to prevent memory permit leaks. */ void release(AsyncDualMemoryLimiterPermit permit); + + /** + * Helper method to acquire permits, execute function, and release permits. + * @param memorySize size in bytes + * @param limitType type of memory limit + * @param isCancelled supplier to check if request should be cancelled + * @param function function to execute with acquired permits + * @param permitAcquireErrorHandler error handler for permit acquisition failures + * @return result of the function + */ + default CompletableFuture withAcquiredPermits(long memorySize, LimitType limitType, + BooleanSupplier isCancelled, + Function> function, + Function> + permitAcquireErrorHandler); + + /** + * Helper method to update permits, execute function, and release permits. + */ + default CompletableFuture withUpdatedPermits(AsyncDualMemoryLimiterPermit initialPermit, + long newMemorySize, + BooleanSupplier isCancelled, + Function> function, + Function> + permitAcquireErrorHandler); +} +``` + +#### AsyncDualMemoryLimiterUtil Helper + +A utility class provides helper methods for common patterns: + +```java +public class AsyncDualMemoryLimiterUtil { + /** + * Acquire permits and write the command as the response to the channel. + * Releases the permits after the response has been written or the write fails. + */ + public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush( + ChannelHandlerContext ctx, + AsyncDualMemoryLimiter dualMemoryLimiter, + BooleanSupplier isCancelled, + BaseCommand command, + Consumer permitAcquireErrorHandler); } ``` #### Integration Points -**1. Heap Memory Limiting (Post-Retrieval)** +**1. Heap Memory Limiting (Post-Retrieval) - Broker** -In `ServerCnx.handleGetTopicsOfNamespace`: +In `ServerCnx.handleGetTopicsOfNamespace`, the implementation uses the helper methods: ```java -// Acquire a fixed amount of permits initially since it's not known how much memory will be used -// This will ensure that the operation continues only after it has the initial permits -// It would be possible to use statistics for initial estimate, but this is simpler and sufficient -maxTopicListInFlightLimiter.acquire(INITIAL_SIZE, AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY) - .thenCompose(initialPermit -> { - getBrokerService().pulsar().getNamespaceService().getListOfUserTopics(namespaceName, mode) - .thenCompose(topics -> { - // Estimate memory after retrieval and update the permits to reflect the actual size - long estimatedSize = topics.stream().mapToInt(String::length).sum(); - return maxTopicListInFlightLimiter - .update(initialPermit, estimatedSize) - .thenApply(permit -> Pair.of(topics, permit)); - }) - .thenAccept(topicsAndPermit -> { - try { - // Process and send response - ... - } finally { - maxTopicListInFlightLimiter.release(topicsAndPermit.getRight()); - } +private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName namespaceName, + long requestId, CommandGetTopicsOfNamespace.Mode mode, + Optional topicsPattern, Optional topicsHash, + Semaphore lookupSemaphore) { + BooleanSupplier isPermitRequestCancelled = () -> !ctx().channel().isActive(); + + // Acquire initial heap permit with 1KB estimate + maxTopicListInFlightLimiter.withAcquiredPermits( + INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, // 1KB + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, + isPermitRequestCancelled, + initialPermits -> { + return getBrokerService().pulsar().getNamespaceService() + .getListOfUserTopics(namespaceName, mode) + .thenAccept(topics -> { + // Calculate actual size + long actualSize = topics.stream().mapToInt(String::length).sum(); + + // Update to actual size + maxTopicListInFlightLimiter.withUpdatedPermits( + initialPermits, actualSize, isPermitRequestCancelled, + permits -> { + // Filter and process topics + List filteredTopics = filterTopics(topics, topicsPattern); + String hash = TopicList.calculateHash(filteredTopics); + boolean hashUnchanged = topicsHash.isPresent() + && topicsHash.get().equals(hash); + + // Send response with direct memory limiting + commandSender.sendGetTopicsOfNamespaceResponse( + filteredTopics, hash, filterTopics, + !hashUnchanged, requestId, + ex -> { + // Handle direct memory permit acquisition failure + log.warn("[{}] Failed to acquire direct memory permits", + remoteAddress, ex); + commandSender.sendErrorResponse(requestId, + ServerError.TooManyRequests, + "Cannot acquire permits for direct memory"); + }); + return CompletableFuture.completedFuture(null); + }, + t -> { + // Handle heap memory update failure + log.warn("[{}] Failed to acquire heap memory permits", + remoteAddress, t); + writeAndFlush(Commands.newError(requestId, + ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + return CompletableFuture.completedFuture(null); + }); + }).whenComplete((__, ___) -> { + lookupSemaphore.release(); + }); + }, + t -> { + // Handle initial heap memory acquisition failure + log.warn("[{}] Failed to acquire initial heap memory permits", + remoteAddress, t); + writeAndFlush(Commands.newError(requestId, + ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + return CompletableFuture.completedFuture(null); }); - ... - // For exceptional paths, initialPermit would need to be released +} ``` -**2. Direct Memory Limiting (Pre-Serialization)** +**2. Direct Memory Limiting (Pre-Serialization) - Broker** -Modified `CommandSender` implementation: +Modified `PulsarCommandSenderImpl`: ```java @Override public void sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, - boolean filtered, boolean changed, long requestId) { - BaseCommand command = Commands.newGetTopicsOfNamespaceResponseCommand(topics, topicsHash, - filtered, changed, requestId); + boolean filtered, boolean changed, long requestId, + Consumer permitAcquireErrorHandler) { + BaseCommand command = Commands.newGetTopicsOfNamespaceResponseCommand( + topics, topicsHash, filtered, changed, requestId); safeIntercept(command, cnx); - acquireMaxTopicListInFlightPermitsAndWriteAndFlush(command); + + // Use utility method to acquire permits, serialize, write, and release + acquireDirectMemoryPermitsAndWriteAndFlush( + cnx.ctx(), maxTopicListInFlightLimiter, + () -> !cnx.isActive(), + command, permitAcquireErrorHandler); } +``` -private void acquireMaxTopicListInFlightPermitsAndWriteAndFlush(BaseCommand command) { +The utility method implementation: + +```java +public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush( + ChannelHandlerContext ctx, + AsyncDualMemoryLimiter dualMemoryLimiter, + BooleanSupplier isCancelled, + BaseCommand command, + Consumer permitAcquireErrorHandler) { + // Calculate serialized size before acquiring permits int serializedSize = command.getSerializedSize(); + // Acquire permits - maxTopicListInFlightLimiter.acquire(serializedSize, AsyncDualMemoryLimiter.LimitType.DIRECT_MEMORY) - .thenAcceptAsync(permits -> { - try { - // Serialize the response - ByteBuf outBuf = Commands.serializeWithPrecalculatedSerializedSize(command, serializedSize); - // Write the response - cnx.ctx().writeAndFlush(outBuf).addListener(future -> { - // Release permits after the response has been written to the socket - maxTopicListInFlightLimiter.release(permits); - }); - } catch (Exception e) { - // Return permits if an exception occurs before writeAndFlush is called successfully - maxTopicListInFlightLimiter.release(permits); - throw e; - } - }, cnx.ctx().executor()); + return dualMemoryLimiter.acquire(serializedSize, + AsyncDualMemoryLimiter.LimitType.DIRECT_MEMORY, + isCancelled) + .whenComplete((permits, t) -> { + if (t != null) { + permitAcquireErrorHandler.accept(t); + return; + } + try { + // Serialize with pre-calculated size + ByteBuf outBuf = Commands.serializeWithPrecalculatedSerializedSize( + command, serializedSize); + + // Write and release permits after completion + ctx.writeAndFlush(outBuf).addListener(future -> { + dualMemoryLimiter.release(permits); + }); + } catch (Exception e) { + // Release permits if exception occurs before writeAndFlush + dualMemoryLimiter.release(permits); + throw e; + } + }).thenAccept(__ -> {}); } ``` -**3. Watch Command Memory Control** +**3. Watch Command Memory Control - Broker** -Similar memory limiting patterns apply to watch commands: +Similar memory limiting patterns apply to watch commands in `TopicListService`: ```java -public void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics) { - BaseCommand command = Commands.newWatchTopicListSuccess(requestId, watcherId, topicsHash, topics); - acquireMaxTopicListInFlightPermitsAndWriteAndFlush(command); -} - -public void sendWatchTopicListUpdate(long watcherId, List newTopics, List deletedTopics, String topicsHash) { - BaseCommand command = Commands.newWatchTopicUpdate(watcherId, newTopics, deletedTopics, topicsHash); - acquireMaxTopicListInFlightPermitsAndWriteAndFlush(command); +public void sendTopicListUpdate(long watcherId, String topicsHash, + List deletedTopics, List newTopics) { + connection.getCommandSender().sendWatchTopicListUpdate( + watcherId, newTopics, deletedTopics, topicsHash, + t -> { + log.warn("[{}] Cannot acquire direct memory tokens for sending topic list update", + connection.toString(), t); + }); } ``` -**4. Proxy Reading Memory Control** - -On the Pulsar Proxy side, the problem is slightly different. The problem occurs when the proxy receives a `CommandGetTopicsOfNamespace` command, forwards it to a broker, and receives a response. The proxy needs to deserialize and serialize the response before sending it to the client. -Memory is allocated for both deserialization and serialization. - -Solving this requires a slight modification to PulsarDecoder. +**4. Proxy Memory Control** -In `PulsarDecoder.channelRead`, it would be necessary to record the size of the incoming message: +On the Pulsar Proxy side in `LookupProxyHandler`: ```java - // Get a buffer that contains the full frame - ByteBuf buffer = (ByteBuf) msg; - try { - // De-serialize the command - int cmdSize = (int) buffer.readUnsignedInt(); - cmd.parseFrom(buffer, cmdSize); -``` +private void internalPerformGetTopicsOfNamespace(long clientRequestId, String namespaceName, + ClientCnx clientCnx, ByteBuf command, + long requestId) { + BooleanSupplier isPermitRequestCancelled = () -> !proxyConnection.ctx().channel().isActive(); + + // Acquire initial heap permit + maxTopicListInFlightLimiter.withAcquiredPermits( + INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, + isPermitRequestCancelled, + initialPermits -> { + return clientCnx.newGetTopicsOfNamespace(command, requestId) + .whenComplete((r, t) -> { + if (t != null) { + log.warn("[{}] Failed to get TopicsOfNamespace {}", + clientAddress, namespaceName, t); + writeAndFlush(Commands.newError(clientRequestId, + getServerError(t), t.getMessage())); + } else { + // Update to actual size + long actualSize = r.getNonPartitionedOrPartitionTopics() + .stream().mapToInt(String::length).sum(); + + maxTopicListInFlightLimiter.withUpdatedPermits( + initialPermits, actualSize, isPermitRequestCancelled, + permits -> { + return handleWritingGetTopicsResponse( + clientRequestId, r, isPermitRequestCancelled); + }, + t2 -> { + log.warn("[{}] Failed to acquire heap memory permits", + clientAddress, t2); + writeAndFlush(Commands.newError(clientRequestId, + ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + return CompletableFuture.completedFuture(null); + }); + } + }).thenApply(__ -> null); + }, + t -> { + log.warn("[{}] Failed to acquire initial heap memory permits", + clientAddress, t); + writeAndFlush(Commands.newError(clientRequestId, + ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + return CompletableFuture.completedFuture(null); + }); +} -It could be modified to store the `cmdSize` in a field instead of a local variable: -```java - protected int cmdSize; -... - // Get a buffer that contains the full frame - ByteBuf buffer = (ByteBuf) msg; - try { - // De-serialize the command - cmdSize = (int) buffer.readUnsignedInt(); - cmd.parseFrom(buffer, cmdSize); +private CompletableFuture handleWritingGetTopicsResponse( + long clientRequestId, GetTopicsResult r, BooleanSupplier isCancelled) { + BaseCommand responseCommand = Commands.newGetTopicsOfNamespaceResponseCommand( + r.getNonPartitionedOrPartitionTopics(), r.getTopicsHash(), + r.isFiltered(), r.isChanged(), clientRequestId); + + return acquireDirectMemoryPermitsAndWriteAndFlush( + proxyConnection.ctx(), maxTopicListInFlightLimiter, + isCancelled, responseCommand, + t -> { + log.warn("[{}] Failed to acquire direct memory permits", + clientAddress, t); + writeAndFlush(Commands.newError(clientRequestId, + ServerError.TooManyRequests, + "Failed due to direct memory limit exceeded")); + }); +} ``` -Changes would be needed to be able to use this serialized size so that it doesn't need to be re-calculated. -`cmdSize` would be added as a field to `GetTopicsResult`: +#### TopicListMemoryLimiter - Metrics Integration -```java -@Override -protected void handleGetTopicsOfNamespaceSuccess(CommandGetTopicsOfNamespaceResponse success) { - checkArgument(state == State.Ready); - - long requestId = success.getRequestId(); - List topics = success.getTopicsList(); +The `TopicListMemoryLimiter` class extends `AsyncDualMemoryLimiterImpl` and adds Prometheus and OpenTelemetry metrics: - - if (log.isDebugEnabled()) { - log.debug("{} Received get topics of namespace success response from server: {} - topics.size: {}", - ctx.channel(), success.getRequestId(), topics.size()); +```java +public class TopicListMemoryLimiter extends AsyncDualMemoryLimiterImpl { + private final CollectorRegistry collectorRegistry; + private final Gauge heapMemoryUsedBytes; + private final Gauge heapMemoryLimitBytes; + // ... other Prometheus metrics + private final ObservableDoubleGauge otelHeapMemoryUsedGauge; + // ... other OpenTelemetry metrics + + public TopicListMemoryLimiter(CollectorRegistry collectorRegistry, + String prometheusPrefix, + Meter openTelemetryMeter, + long maxHeapMemory, int maxHeapQueueSize, + long heapTimeoutMillis, long maxDirectMemory, + int maxDirectQueueSize, long directTimeoutMillis) { + super(maxHeapMemory, maxHeapQueueSize, heapTimeoutMillis, + maxDirectMemory, maxDirectQueueSize, directTimeoutMillis); + + // Initialize Prometheus metrics + this.heapMemoryUsedBytes = Gauge.build( + prometheusPrefix + "topic_list_heap_memory_used_bytes", + "Current heap memory used by topic listings") + .create() + .setChild(new Gauge.Child() { + @Override + public double get() { + return getLimiter(LimitType.HEAP_MEMORY).getAcquiredPermits(); + } + }) + .register(collectorRegistry); + + // Initialize OpenTelemetry metrics + this.otelHeapMemoryUsedGauge = openTelemetryMeter + .gaugeBuilder("topic.list.heap.memory.used") + .setUnit("By") + .setDescription("Current heap memory used by topic listings") + .buildWithCallback(measurement -> { + measurement.record(getLimiter(LimitType.HEAP_MEMORY).getAcquiredPermits()); + }); + + // ... initialize other metrics } - - CompletableFuture requestFuture = - (CompletableFuture) pendingRequests.remove(requestId); - if (requestFuture != null) { - requestFuture.complete(new GetTopicsResult(topics, - success.hasTopicsHash() ? success.getTopicsHash() : null, - success.isFiltered(), - success.isChanged(), - // Store cmdSize in the GetTopicsResult <---- - cmdSize)); - } else { - duplicatedResponseCounter.incrementAndGet(); - log.warn("{} Received unknown request id from server: {}", ctx.channel(), success.getRequestId()); + + @Override + protected void recordHeapWaitTime(long waitTimeNanos) { + if (waitTimeNanos == Long.MAX_VALUE) { + heapTimeoutTotal.inc(); + otelHeapTimeoutTotal.add(1); + } else { + heapWaitTimeMs.observe(TimeUnit.NANOSECONDS.toMillis(waitTimeNanos)); + otelHeapWaitTime.record(waitTimeNanos / 1_000_000_000.0d); + } + } + + @Override + protected void recordDirectWaitTime(long waitTimeNanos) { + // Similar implementation for direct memory } } ``` -The limiter would be integrated into `LookupProxyHandler`'s `performGetTopicsOfNamespace` in this way: - -```java - proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { - // Connected to backend broker - long requestId = proxyConnection.newRequestId(); - ByteBuf command; - command = Commands.newGetTopicsOfNamespaceRequest(namespaceName, requestId, mode, - topicsPattern, topicsHash); - // Acquire a fixed amount of permits initially since it's not known how much memory will be used - // This will ensure that the operation continues only after it has the initial permits - // It would be possible to use statistics for initial estimate, but this is simpler and sufficient - maxTopicListInFlightLimiter.acquire(INITIAL_SIZE, AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY) - .thenCompose(initialPermit -> { - clientCnx.newGetTopicsOfNamespace(command, requestId).whenComplete((r, t) -> { - if (t != null) { - maxTopicListInFlightLimiter.release(initialPermit); - log.warn("[{}] Failed to get TopicsOfNamespace {}: {}", - clientAddress, namespaceName, t.getMessage()); - writeAndFlush( - Commands.newError(clientRequestId, getServerError(t), t.getMessage())); - } else { - // Update the initial permits to reflect the actual size of the response - maxTopicListInFlightLimiter.update(initialPermit, r.getSerializedSize()) - .thenCompose(heapPermit -> { - // Acquire a direct memory permit for serialization - maxTopicListInFlightLimiter.acquire(r.getSerializedSize(), AsyncDualMemoryLimiter.LimitType.DIRECT_MEMORY) - .thenAccept(directPermit -> { - proxyConnection.ctx().writeAndFlush( - Commands.newGetTopicsOfNamespaceResponse(r.getNonPartitionedOrPartitionTopics(), - r.getTopicsHash(), r.isFiltered(), - r.isChanged(), clientRequestId) - ).addListener(future -> { - // Release permits after the response has been written to the socket - maxTopicListInFlightLimiter.release(heapPermit); - maxTopicListInFlightLimiter.release(directPermit); - }); - }) // Add exception handling for releasing directPermit - }); // Add exception handling for releasing heapPermit - } - }); - }); - proxyConnection.getConnectionPool().releaseConnection(clientCnx); - }).exceptionally(ex -> { -``` - ### Public-facing Changes #### Configuration @@ -404,21 +572,21 @@ maxTopicListInFlightDirectMemSizeMB=100 maxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis=25000 # Maximum queue size for heap memory permit requests -# Default: 1000 (prevent unbounded queueing) -maxTopicListInFlightHeapMemSizePermitsAcquireQueueSize=1000 +# Default: 10000 (prevent unbounded queueing) +maxTopicListInFlightHeapMemSizePermitsAcquireQueueSize=10000 # Timeout for acquiring direct memory permits (milliseconds) # Default: 25000 (25 seconds) maxTopicListInFlightDirectMemSizePermitsAcquireTimeoutMillis=25000 # Maximum queue size for direct memory permit requests -# Default: 1000 (prevent unbounded queueing) -maxTopicListInFlightDirectMemSizePermitsAcquireQueueSize=1000 +# Default: 10000 (prevent unbounded queueing) +maxTopicListInFlightDirectMemSizePermitsAcquireQueueSize=10000 ``` #### Metrics -New metrics under `pulsar_broker_topic_list_`/`pulsar_proxy_topic_list` prefix, complementing existing memory metrics: +New metrics under `pulsar_broker_topic_list_`/`pulsar_proxy_topic_list_` prefix, complementing existing memory metrics: | Metric Name | Type | Description | Labels | |------------|------|-------------|--------| @@ -430,11 +598,13 @@ New metrics under `pulsar_broker_topic_list_`/`pulsar_proxy_topic_list` prefix, | `heap_queue_max_size` | Gauge | Maximum heap memory limiter queue size | `cluster` | | `direct_queue_size` | Gauge | Current direct memory limiter queue size | `cluster` | | `direct_queue_max_size` | Gauge | Maximum direct memory limiter queue size | `cluster` | -| `heap_wait_time_ms` | Histogram | Wait time for heap memory permits | `cluster` | -| `direct_wait_time_ms` | Histogram | Wait time for direct memory permits | `cluster` | +| `heap_wait_time_ms` | Summary | Wait time for heap memory permits (quantiles: 0.5, 0.95, 0.99, 1.0) | `cluster` | +| `direct_wait_time_ms` | Summary | Wait time for direct memory permits (quantiles: 0.5, 0.95, 0.99, 1.0) | `cluster` | | `heap_timeout_total` | Counter | Total heap memory permit timeouts | `cluster` | | `direct_timeout_total` | Counter | Total direct memory permit timeouts | `cluster` | +OpenTelemetry equivalents are also provided with similar naming under the `topic.list.*` namespace. + #### Public API No changes to REST API. @@ -448,20 +618,20 @@ No protocol changes. Existing commands continue to work with added server-side m Operators should monitor the following metrics alongside existing memory management metrics and set up alerts: 1. **Memory Utilization Alert**: - - Alert when `heap_memory_used_bytes / heap_memory_limit_bytes > 0.8` - - Indicates the need to increase limits or investigate namespace growth + - Alert when `heap_memory_used_bytes / heap_memory_limit_bytes > 0.8` + - Indicates the need to increase limits or investigate namespace growth 2. **Queue Saturation Alert**: - - Alert when `heap_queue_size / heap_queue_max_size > 0.9` - - Indicates sustained memory pressure requiring capacity adjustment + - Alert when `heap_queue_size / heap_queue_max_size > 0.9` + - Indicates sustained memory pressure requiring capacity adjustment 3. **Timeout Rate Alert**: - - Alert when `rate(heap_timeout_total[5m]) > 1` - - Indicates clients experiencing failures due to memory-based flow control + - Alert when `rate(heap_timeout_total[5m]) > 1` + - Indicates clients experiencing failures due to memory-based flow control 4. **P99 Wait Time Alert**: - - Alert when `heap_wait_time_ms{quantile="0.99"} > 10000` - - Indicates degraded client experience due to memory pressure + - Alert when `heap_wait_time_ms{quantile="0.99"} > 10000` + - Indicates degraded client experience due to memory pressure These alerts should be configured alongside existing memory alerts for `managedLedgerCacheSizeMB`, `maxMessagePublishBufferSizeInMB`, and other memory limits to provide comprehensive memory management visibility. @@ -473,30 +643,22 @@ The memory limiting mechanism introduces new denial-of-service protections: 2. **Fair Queueing**: The queue size limits prevent bad clients from monopolizing memory permits and blocking legitimate requests. -3. **Multi-tenancy Isolation**: Consider per-tenant memory limits in future iterations to prevent one tenant from consuming all available topic listing memory permits, similar to how other memory limits could benefit from tenant isolation. +3. **Cancellation Support**: The implementation includes cancellation support to prevent resource waste when clients disconnect, preventing malicious clients from queueing many requests and then disconnecting. + +4. **Multi-tenancy Isolation**: Consider per-tenant memory limits in future iterations to prevent one tenant from consuming all available topic listing memory permits, similar to how other memory limits could benefit from tenant isolation. ## Backward & Forward Compatibility ### Upgrade -1. The feature can be disabled setting the limits set to 0 initially to maintain full compatibility -2. After upgrade, gradually enable memory limits: - ```bash - # Start with high limits to understand current usage - pulsar-admin brokers update-dynamic-config --config maxTopicListInFlightHeapMemSizeMB --value 512 - pulsar-admin brokers update-dynamic-config --config maxTopicListInFlightDirectMemSizeMB --value 512 - - # Monitor metrics and adjust downward based on actual usage patterns - pulsar-admin brokers update-dynamic-config --config maxTopicListInFlightHeapMemSizeMB --value 200 - pulsar-admin brokers update-dynamic-config --config maxTopicListInFlightDirectMemSizeMB --value 200 - pulsar-admin brokers update-dynamic-config --config maxTopicListInFlightHeapMemSizeMB --value 100 - pulsar-admin brokers update-dynamic-config --config maxTopicListInFlightDirectMemSizeMB --value 100 - ``` +1. The feature is enabled by default with reasonable limits (100MB each for heap and direct memory) +2. After upgrade monitor metrics 3. No client changes are required ### Downgrade / Rollback - No changes required +- If downgrading, the memory limiting behavior will be lost but no functional issues will occur ### Pulsar Geo-Replication Upgrade & Downgrade/Rollback Considerations From a76e16c16c1033576886950b3179c8624bd5a5f3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 16:55:06 +0300 Subject: [PATCH 07/75] Remove dynamic configuration for topic list configuration --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 614292b18ef28..b42428a33474d 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -1387,42 +1387,36 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece private int maxConcurrentLookupRequest = 50000; @FieldContext( - dynamic = true, category = CATEGORY_SERVER, doc = "Maximum heap memory for inflight topic list operations (MB).\n" + "Default: 100 MB (supports ~1M topic names assuming 100 bytes each)") private int maxTopicListInFlightHeapMemSizeMB = 100; @FieldContext( - dynamic = true, category = CATEGORY_SERVER, doc = "Maximum direct memory for inflight topic list responses (MB).\n" + "Default: 100 MB (network buffers for serialized responses)") private int maxTopicListInFlightDirectMemSizeMB = 100; @FieldContext( - dynamic = true, category = CATEGORY_SERVER, doc = "Timeout for acquiring heap memory permits (milliseconds).\n" + "Default: 25000 (25 seconds)") private int maxTopicListInFlightHeapMemSizePermitsAcquireTimeoutMillis = 25000; @FieldContext( - dynamic = true, category = CATEGORY_SERVER, doc = "Maximum queue size for heap memory permit requests.\n" + "Default: 10000 (prevent unbounded queueing)") private int maxTopicListInFlightHeapMemSizePermitsAcquireQueueSize = 10000; @FieldContext( - dynamic = true, category = CATEGORY_SERVER, doc = "Timeout for acquiring direct memory permits (milliseconds).\n" + "Default: 25000 (25 seconds)") private int maxTopicListInFlightDirectMemSizePermitsAcquireTimeoutMillis = 25000; @FieldContext( - dynamic = true, category = CATEGORY_SERVER, doc = "Maximum queue size for direct memory permit requests.\n" + "Default: 10000 (prevent unbounded queueing)") From 22728d87f5a5551bc46581b750be1de81f375378 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 17:58:17 +0300 Subject: [PATCH 08/75] Ignore registration and unregistration exceptions in tests --- .../TopicListMemoryLimiter.java | 95 +++++++++++-------- 1 file changed, 55 insertions(+), 40 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java index 2a3fadbd0f291..d6a91f4a175ce 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java @@ -25,6 +25,7 @@ import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.api.metrics.ObservableDoubleGauge; import io.opentelemetry.api.metrics.ObservableLongUpDownCounter; +import io.prometheus.client.Collector; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; @@ -73,7 +74,7 @@ public TopicListMemoryLimiter(CollectorRegistry collectorRegistry, String promet AsyncSemaphore heapMemoryLimiter = getLimiter(LimitType.HEAP_MEMORY); AsyncSemaphore directMemoryLimiter = getLimiter(LimitType.DIRECT_MEMORY); - this.heapMemoryUsedBytes = Gauge.build(prometheusPrefix + "topic_list_heap_memory_used_bytes", + this.heapMemoryUsedBytes = register(Gauge.build(prometheusPrefix + "topic_list_heap_memory_used_bytes", "Current heap memory used by topic listings") .create() .setChild(new Gauge.Child() { @@ -81,8 +82,7 @@ public TopicListMemoryLimiter(CollectorRegistry collectorRegistry, String promet public double get() { return heapMemoryLimiter.getAcquiredPermits(); } - }) - .register(collectorRegistry); + })); this.otelHeapMemoryUsedGauge = openTelemetryMeter.gaugeBuilder("topic.list.heap.memory.used") .setUnit("By") .setDescription("Current heap memory used by topic listings") @@ -90,9 +90,9 @@ public double get() { observableDoubleMeasurement.record(heapMemoryLimiter.getAcquiredPermits()); }); - this.heapMemoryLimitBytes = Gauge.build(prometheusPrefix + "topic_list_heap_memory_limit_bytes", + this.heapMemoryLimitBytes = register(Gauge.build(prometheusPrefix + "topic_list_heap_memory_limit_bytes", "Configured heap memory limit") - .create().register(collectorRegistry); + .create()); this.heapMemoryLimitBytes.set(maxHeapMemory); this.otelHeapMemoryLimitGauge = openTelemetryMeter.gaugeBuilder("topic.list.heap.memory.limit") .setUnit("By") @@ -100,7 +100,7 @@ public double get() { .build(); this.otelHeapMemoryLimitGauge.set(maxHeapMemory); - this.directMemoryUsedBytes = Gauge.build(prometheusPrefix + "topic_list_direct_memory_used_bytes", + this.directMemoryUsedBytes = register(Gauge.build(prometheusPrefix + "topic_list_direct_memory_used_bytes", "Current direct memory used by topic listings") .create() .setChild(new Gauge.Child() { @@ -108,8 +108,7 @@ public double get() { public double get() { return directMemoryLimiter.getAcquiredPermits(); } - }) - .register(collectorRegistry); + })); this.otelDirectMemoryUsedGauge = openTelemetryMeter.gaugeBuilder("topic.list.direct.memory.used") .setUnit("By") .setDescription("Current direct memory used by topic listings") @@ -117,9 +116,9 @@ public double get() { observableDoubleMeasurement.record(directMemoryLimiter.getAcquiredPermits()); }); - this.directMemoryLimitBytes = Gauge.build(prometheusPrefix + "topic_list_direct_memory_limit_bytes", + this.directMemoryLimitBytes = register(Gauge.build(prometheusPrefix + "topic_list_direct_memory_limit_bytes", "Configured direct memory limit") - .create().register(collectorRegistry); + .create()); this.directMemoryLimitBytes.set(maxDirectMemory); this.otelDirectMemoryLimitGauge = openTelemetryMeter.gaugeBuilder("topic.list.direct.memory.limit") .setUnit("By") @@ -127,7 +126,7 @@ public double get() { .build(); this.otelDirectMemoryLimitGauge.set(maxHeapMemory); - this.heapQueueSize = Gauge.build(prometheusPrefix + "topic_list_heap_queue_size", + this.heapQueueSize = register(Gauge.build(prometheusPrefix + "topic_list_heap_queue_size", "Current heap memory limiter queue size") .create() .setChild(new Gauge.Child() { @@ -135,8 +134,7 @@ public double get() { public double get() { return heapMemoryLimiter.getQueueSize(); } - }) - .register(collectorRegistry); + })); this.otelHeapQueueSize = openTelemetryMeter .upDownCounterBuilder("topic.list.heap.queue.size") .setDescription("Current heap memory limiter queue size") @@ -145,9 +143,9 @@ public double get() { observableLongMeasurement.record(heapMemoryLimiter.getQueueSize()); }); - this.heapQueueMaxSize = Gauge.build(prometheusPrefix + "topic_list_heap_queue_max_size", + this.heapQueueMaxSize = register(Gauge.build(prometheusPrefix + "topic_list_heap_queue_max_size", "Maximum heap memory limiter queue size") - .create().register(collectorRegistry); + .create()); this.heapQueueMaxSize.set(maxHeapQueueSize); LongGauge otelHeapQueueMaxSize = openTelemetryMeter .gaugeBuilder("topic.list.heap.queue.max.size") @@ -157,7 +155,7 @@ public double get() { .build(); otelHeapQueueMaxSize.set(maxHeapQueueSize); - this.directQueueSize = Gauge.build(prometheusPrefix + "topic_list_direct_queue_size", + this.directQueueSize = register(Gauge.build(prometheusPrefix + "topic_list_direct_queue_size", "Current direct memory limiter queue size") .create() .setChild(new Gauge.Child() { @@ -165,8 +163,7 @@ public double get() { public double get() { return directMemoryLimiter.getQueueSize(); } - }) - .register(collectorRegistry); + })); this.otelDirectQueueSize = openTelemetryMeter .upDownCounterBuilder("topic.list.direct.queue.size") .setDescription("Current direct memory limiter queue size") @@ -175,9 +172,9 @@ public double get() { observableLongMeasurement.record(directMemoryLimiter.getQueueSize()); }); - this.directQueueMaxSize = Gauge.build(prometheusPrefix + "topic_list_direct_queue_max_size", + this.directQueueMaxSize = register(Gauge.build(prometheusPrefix + "topic_list_direct_queue_max_size", "Maximum direct memory limiter queue size") - .create().register(collectorRegistry); + .create().register(collectorRegistry)); this.directQueueMaxSize.set(maxDirectQueueSize); LongGauge otelDirectQueueMaxSize = openTelemetryMeter .gaugeBuilder("topic.list.direct.queue.max.size") @@ -187,47 +184,65 @@ public double get() { .build(); otelDirectQueueMaxSize.set(maxDirectQueueSize); - this.heapWaitTimeMs = Summary.build(prometheusPrefix + "topic_list_heap_wait_time_ms", + this.heapWaitTimeMs = register(Summary.build(prometheusPrefix + "topic_list_heap_wait_time_ms", "Wait time for heap memory permits") .quantile(0.50, 0.01) .quantile(0.95, 0.01) .quantile(0.99, 0.01) .quantile(1, 0.01) - .create().register(collectorRegistry); + .create()); this.otelHeapWaitTime = openTelemetryMeter.histogramBuilder("topic.list.heap.wait.time.ms") .setUnit("s") .setDescription("Wait time for heap memory permits") .build(); - this.directWaitTimeMs = Summary.build(prometheusPrefix + "topic_list_direct_wait_time_ms", + this.directWaitTimeMs = register(Summary.build(prometheusPrefix + "topic_list_direct_wait_time_ms", "Wait time for direct memory permits") .quantile(0.50, 0.01) .quantile(0.95, 0.01) .quantile(0.99, 0.01) .quantile(1, 0.01) - .create().register(collectorRegistry); + .create()); this.otelDirectWaitTime = openTelemetryMeter.histogramBuilder("topic.list.direct.wait.time.ms") .setUnit("s") .setDescription("Wait time for direct memory permits") .build(); - this.heapTimeoutTotal = Counter.build(prometheusPrefix + "topic_list_heap_timeout_total", + this.heapTimeoutTotal = register(Counter.build(prometheusPrefix + "topic_list_heap_timeout_total", "Total heap memory permit timeouts") - .create().register(collectorRegistry); + .create()); this.otelHeapTimeoutTotal = openTelemetryMeter.counterBuilder("topic.list.heap.timeout.total") .setDescription("Total heap memory permit timeouts") .setUnit("1") .build(); - this.directTimeoutTotal = Counter.build(prometheusPrefix + "topic_list_direct_timeout_total", + this.directTimeoutTotal = register(Counter.build(prometheusPrefix + "topic_list_direct_timeout_total", "Total direct memory permit timeouts") - .create().register(collectorRegistry); + .create()); this.otelDirectTimeoutTotal = openTelemetryMeter.counterBuilder("topic.list.direct.timeout.total") .setDescription("Total direct memory permit timeouts") .setUnit("1") .build(); } + private T register(T collector) { + try { + collectorRegistry.register(collector); + } catch (Exception e) { + // ignore exception when registering a collector that is already registered + } + return collector; + } + + private void unregister(Collector collector) { + try { + collectorRegistry.unregister(collector); + } catch (Exception e) { + // ignore exception when unregistering a collector that is not registered + } + } + + @Override protected void recordHeapWaitTime(long waitTimeNanos) { if (waitTimeNanos == Long.MAX_VALUE) { @@ -253,18 +268,18 @@ protected void recordDirectWaitTime(long waitTimeNanos) { @Override public void close() { super.close(); - collectorRegistry.unregister(heapMemoryUsedBytes); - collectorRegistry.unregister(heapMemoryLimitBytes); - collectorRegistry.unregister(directMemoryUsedBytes); - collectorRegistry.unregister(directMemoryLimitBytes); - collectorRegistry.unregister(heapQueueSize); - collectorRegistry.unregister(heapQueueMaxSize); - collectorRegistry.unregister(directQueueSize); - collectorRegistry.unregister(directQueueMaxSize); - collectorRegistry.unregister(heapWaitTimeMs); - collectorRegistry.unregister(directWaitTimeMs); - collectorRegistry.unregister(heapTimeoutTotal); - collectorRegistry.unregister(directTimeoutTotal); + unregister(heapMemoryUsedBytes); + unregister(heapMemoryLimitBytes); + unregister(directMemoryUsedBytes); + unregister(directMemoryLimitBytes); + unregister(heapQueueSize); + unregister(heapQueueMaxSize); + unregister(directQueueSize); + unregister(directQueueMaxSize); + unregister(heapWaitTimeMs); + unregister(directWaitTimeMs); + unregister(heapTimeoutTotal); + unregister(directTimeoutTotal); otelHeapMemoryUsedGauge.close(); otelDirectMemoryUsedGauge.close(); otelHeapQueueSize.close(); From 93bd9d3db88909975844173c9d9bec6c36172f8a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 18:01:52 +0300 Subject: [PATCH 09/75] Log at debug level --- .../broker/topiclistlimit/TopicListMemoryLimiter.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java index d6a91f4a175ce..debe18fafa6d7 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java @@ -31,12 +31,14 @@ import io.prometheus.client.Gauge; import io.prometheus.client.Summary; import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.semaphore.AsyncSemaphore; /** * Topic list memory limiter that exposes both Prometheus metrics and OpenTelemetry metrics. */ +@Slf4j public class TopicListMemoryLimiter extends AsyncDualMemoryLimiterImpl { private final CollectorRegistry collectorRegistry; private final Gauge heapMemoryUsedBytes; @@ -230,6 +232,9 @@ private T register(T collector) { collectorRegistry.register(collector); } catch (Exception e) { // ignore exception when registering a collector that is already registered + if (log.isDebugEnabled()) { + log.debug("Failed to register Prometheus collector {}", collector, e); + } } return collector; } @@ -239,6 +244,9 @@ private void unregister(Collector collector) { collectorRegistry.unregister(collector); } catch (Exception e) { // ignore exception when unregistering a collector that is not registered + if (log.isDebugEnabled()) { + log.debug("Failed to unregister Prometheus collector {}", collector, e); + } } } From d9d06cc9b5d43361d35ef5c1b4e101a9e6741b0a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 18:55:25 +0300 Subject: [PATCH 10/75] Remove extra register --- .../pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java index debe18fafa6d7..2cd2412811b51 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java @@ -176,7 +176,7 @@ public double get() { this.directQueueMaxSize = register(Gauge.build(prometheusPrefix + "topic_list_direct_queue_max_size", "Maximum direct memory limiter queue size") - .create().register(collectorRegistry)); + .create()); this.directQueueMaxSize.set(maxDirectQueueSize); LongGauge otelDirectQueueMaxSize = openTelemetryMeter .gaugeBuilder("topic.list.direct.queue.max.size") From a8d17d9758f4772fe22b205163d3a5703701e401 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 20:18:37 +0300 Subject: [PATCH 11/75] Fix test --- .../proxy/server/InvalidProxyConfigForAuthorizationTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/InvalidProxyConfigForAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/InvalidProxyConfigForAuthorizationTest.java index e97c355ebe5ae..1bd6e712c9406 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/InvalidProxyConfigForAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/InvalidProxyConfigForAuthorizationTest.java @@ -32,6 +32,7 @@ void startupShouldFailWhenAuthorizationIsEnabledWithoutAuthentication() throws E ProxyConfiguration proxyConfiguration = new ProxyConfiguration(); proxyConfiguration.setAuthorizationEnabled(true); proxyConfiguration.setAuthenticationEnabled(false); + proxyConfiguration.setClusterName("test"); try (ProxyService proxyService = new ProxyService(proxyConfiguration, Mockito.mock(AuthenticationService.class), Mockito.mock(Authentication.class))) { proxyService.start(); From 0d745659bfa517f80b61896ba1b2622d360e1d4a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 20:24:04 +0300 Subject: [PATCH 12/75] Fix test --- .../common/semaphore/AsyncDualMemoryLimiterUtilTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java index 947812bdc2267..e998cce302591 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java @@ -27,7 +27,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -44,6 +43,7 @@ import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter.AsyncDualMemoryLimiterPermit; import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter.LimitType; import org.apache.pulsar.common.semaphore.AsyncSemaphore.PermitAcquireCancelledException; +import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -388,7 +388,7 @@ public void testAcquireDirectMemoryPermitsAndWriteAndFlushCancelled() throws Exc limiter.release(permits); - assertFalse(errorHandlerCalled.get()); + Awaitility.await().untilAsserted(() -> assertTrue(errorHandlerCalled.get())); verify(ctx, never()).writeAndFlush(any(ByteBuf.class)); assertTrue(result.isCompletedExceptionally()); assertThatThrownBy(() -> result.get()).hasCauseInstanceOf(PermitAcquireCancelledException.class); From 225cb75a75502e899b5ef72f56dedde6d60fa42d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 22:18:49 +0300 Subject: [PATCH 13/75] Update pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java index 2cd2412811b51..7b2d78b2c935d 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java @@ -126,7 +126,7 @@ public double get() { .setUnit("By") .setDescription("Configured direct memory limit") .build(); - this.otelDirectMemoryLimitGauge.set(maxHeapMemory); + this.otelDirectMemoryLimitGauge.set(maxDirectMemory); this.heapQueueSize = register(Gauge.build(prometheusPrefix + "topic_list_heap_queue_size", "Current heap memory limiter queue size") From 74c09efaba45444bbef5553ca3d5bdc6e5b812a9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 22:19:25 +0300 Subject: [PATCH 14/75] Update pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java index e998cce302591..8be79261f819e 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file From 723ccc60f54fefa8993b9b9e4a7912826e563b76 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 10 Oct 2025 22:19:37 +0300 Subject: [PATCH 15/75] Update pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java index 6a079cc192e6b..fd23d95151cb0 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImplTest.java @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file From 858e6ddc6deb2d4d06da96096c9c6a75150eca15 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 13 Oct 2025 22:17:41 +0300 Subject: [PATCH 16/75] Remove redundant files --- .../util/memory/AsyncDualMemoryLimiter.java | 61 ------------------- .../memory/AsyncDualMemoryLimiterPermit.java | 47 -------------- .../common/util/memory/AsyncSemaphore.java | 52 ---------------- .../util/memory/AsyncSemaphorePermit.java | 40 ------------ .../memory/AsyncSemaphorePermitResult.java | 47 -------------- 5 files changed, 247 deletions(-) delete mode 100644 org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiter.java delete mode 100644 org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiterPermit.java delete mode 100644 org/apache/pulsar/common/util/memory/AsyncSemaphore.java delete mode 100644 org/apache/pulsar/common/util/memory/AsyncSemaphorePermit.java delete mode 100644 org/apache/pulsar/common/util/memory/AsyncSemaphorePermitResult.java diff --git a/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiter.java b/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiter.java deleted file mode 100644 index 6260dd0fdffce..0000000000000 --- a/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiter.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util.memory; - -import java.util.concurrent.CompletableFuture; - -/** - * Interface for asynchronous dual memory limiting. - * Manages limits for both heap and direct memory usage. - */ -public interface AsyncDualMemoryLimiter { - - /** - * Enum representing the type of memory limit. - */ - enum LimitType { - HEAP_MEMORY, // For heap memory allocation - DIRECT_MEMORY // For direct memory allocation - } - - /** - * Acquire memory permits asynchronously. - * - * @param memorySize the amount of memory to acquire in bytes - * @param limitType the type of memory limit - * @return CompletableFuture containing the memory permit - */ - CompletableFuture acquire(long memorySize, LimitType limitType); - - /** - * Update an existing permit with a new memory size. - * - * @param permit the existing permit to update - * @param newMemorySize the new memory size in bytes - * @return CompletableFuture containing the updated permit - */ - CompletableFuture update(AsyncDualMemoryLimiterPermit permit, long newMemorySize); - - /** - * Release a memory permit back to the limiter. - * - * @param permit the permit to release - */ - void release(AsyncDualMemoryLimiterPermit permit); -} diff --git a/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiterPermit.java b/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiterPermit.java deleted file mode 100644 index 043f2dcf48188..0000000000000 --- a/org/apache/pulsar/common/util/memory/AsyncDualMemoryLimiterPermit.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util.memory; - -/** - * Represents a permit acquired from an AsyncDualMemoryLimiter. - * Contains information about the acquired memory allocation. - */ -public interface AsyncDualMemoryLimiterPermit { - - /** - * Get the memory size allocated by this permit. - * - * @return the memory size in bytes - */ - long getMemorySize(); - - /** - * Get the limit type associated with this permit. - * - * @return the limit type - */ - AsyncDualMemoryLimiter.LimitType getLimitType(); - - /** - * Check if this permit is still valid. - * - * @return true if the permit is valid, false otherwise - */ - boolean isValid(); -} diff --git a/org/apache/pulsar/common/util/memory/AsyncSemaphore.java b/org/apache/pulsar/common/util/memory/AsyncSemaphore.java deleted file mode 100644 index 38db6fda79727..0000000000000 --- a/org/apache/pulsar/common/util/memory/AsyncSemaphore.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util.memory; - -import java.util.concurrent.CompletableFuture; - -/** - * Interface for asynchronous semaphore operations. - * Used for managing concurrent access to limited resources. - */ -public interface AsyncSemaphore { - - /** - * Acquire permits asynchronously. - * - * @param permits the number of permits to acquire - * @return CompletableFuture containing the permit result - */ - CompletableFuture acquire(long permits); - - /** - * Update an existing permit with a new permit count. - * - * @param permit the existing permit to update - * @param newPermits the new number of permits - * @return CompletableFuture containing the updated permit - */ - CompletableFuture update(AsyncSemaphorePermit permit, long newPermits); - - /** - * Release a permit back to the semaphore. - * - * @param permit the permit to release - */ - void release(AsyncSemaphorePermit permit); -} diff --git a/org/apache/pulsar/common/util/memory/AsyncSemaphorePermit.java b/org/apache/pulsar/common/util/memory/AsyncSemaphorePermit.java deleted file mode 100644 index 3c176f8bdd6e4..0000000000000 --- a/org/apache/pulsar/common/util/memory/AsyncSemaphorePermit.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util.memory; - -/** - * Represents a permit acquired from an AsyncSemaphore. - * Contains information about the acquired permits. - */ -public interface AsyncSemaphorePermit { - - /** - * Get the number of permits held by this permit. - * - * @return the number of permits - */ - long getPermits(); - - /** - * Check if this permit is still valid. - * - * @return true if the permit is valid, false otherwise - */ - boolean isValid(); -} diff --git a/org/apache/pulsar/common/util/memory/AsyncSemaphorePermitResult.java b/org/apache/pulsar/common/util/memory/AsyncSemaphorePermitResult.java deleted file mode 100644 index d254ccddad87f..0000000000000 --- a/org/apache/pulsar/common/util/memory/AsyncSemaphorePermitResult.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util.memory; - -/** - * Represents the result of an asynchronous semaphore acquisition. - * Contains either a permit or information about why the acquisition failed. - */ -public interface AsyncSemaphorePermitResult { - - /** - * Get the acquired permit if successful. - * - * @return the permit, or null if acquisition failed - */ - AsyncSemaphorePermit getPermit(); - - /** - * Check if the permit acquisition was successful. - * - * @return true if acquisition was successful, false otherwise - */ - boolean isSuccess(); - - /** - * Get the error message if acquisition failed. - * - * @return the error message, or null if acquisition was successful - */ - String getErrorMessage(); -} From 64590d5a72245aa510001d1f22c26c6c35f22500 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 13 Oct 2025 22:22:33 +0300 Subject: [PATCH 17/75] Address review comment: release lookupSemaphore --- .../main/java/org/apache/pulsar/broker/service/ServerCnx.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 307134b07f7af..355f4b1b6614d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -2645,6 +2645,7 @@ private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName remoteAddress, t.getMessage()); writeAndFlush(Commands.newError(requestId, ServerError.TooManyRequests, "Failed due to heap memory limit exceeded")); + lookupSemaphore.release(); return CompletableFuture.completedFuture(null); }); } From aa8d57baa084b334be271cf21de6c6ec7c1ac78e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 13 Oct 2025 22:32:45 +0300 Subject: [PATCH 18/75] Improve javadoc --- .../semaphore/AsyncDualMemoryLimiter.java | 14 ++++++++-- .../semaphore/AsyncDualMemoryLimiterUtil.java | 28 +++++++++++++------ .../common/semaphore/AsyncSemaphore.java | 22 +++++++++++++-- 3 files changed, 52 insertions(+), 12 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java index cdb3c85ebb2a4..59be8db89239a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiter.java @@ -36,6 +36,10 @@ enum LimitType { * Returned future completes when memory permits are available. * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full + * + * @param memorySize the size of memory to acquire permits for + * @param limitType the type of memory limit (HEAP_MEMORY or DIRECT_MEMORY) + * @param isCancelled supplier that returns true if acquisition should be cancelled * @return CompletableFuture that completes with permit when available */ CompletableFuture acquire(long memorySize, LimitType limitType, @@ -48,16 +52,22 @@ CompletableFuture acquire(long memorySize, LimitTy * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full * The provided permit is released when the permits are successfully acquired and the returned updated * permit replaces the old instance. + * + * @param permit the previously acquired permit to update + * @param newMemorySize the new memory size to update to + * @param isCancelled supplier that returns true if update should be cancelled * @return CompletableFuture that completes with permit when available */ CompletableFuture update(AsyncDualMemoryLimiterPermit permit, long newMemorySize, BooleanSupplier isCancelled); + /** * Release previously acquired permit. * Must be called to prevent memory permit leaks. + * + * @param permit the permit to release */ void release(AsyncDualMemoryLimiterPermit permit); - /** * Execute the specified function with acquired permits and release the permits after the returned future completes. * @param memorySize memory size to acquire permits for @@ -95,7 +105,7 @@ default CompletableFuture withUpdatedPermits(AsyncDualMemoryLimiterPermit } /** - * Represents a permit for memory limiting that can be updated or released. + * Represents an acquired permit for memory limiting that can be updated or released. */ interface AsyncDualMemoryLimiterPermit { long getPermits(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java index 1b4d37ca60737..af9870437f4d2 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java @@ -33,6 +33,17 @@ @UtilityClass public class AsyncDualMemoryLimiterUtil { + /** + * Execute a function with acquired permits and ensure permits are released after completion. + * This method handles the lifecycle of permits - acquisition, usage, and release, including error cases. + * + * @param permitsFuture Future that will complete with the required permits + * @param function Function to execute once permits are acquired that returns a CompletableFuture + * @param permitAcquireErrorHandler Handler for permit acquisition errors that returns a CompletableFuture + * @param releaser Consumer that handles releasing the permits + * @param The type of result returned by the function + * @return CompletableFuture that completes with the result of the function execution + */ public static CompletableFuture withPermitsFuture( CompletableFuture permitsFuture, @@ -67,13 +78,15 @@ public static CompletableFuture withPermitsFuture( } /** - * Acquire permits and write the command as the response to the channel. - * Releases the permits after the response has been written to the socket or the write fails. + * Acquires permits and writes the command as a response to the channel. + * Releases the permits after the response has been written to the socket or if the write fails. * - * @param ctx the channel handler context. - * @param dualMemoryLimiter the memory limiter to acquire permits from. - * @param command the command to write to the channel. - * @return a future that completes when the command has been written to the channel's outbound buffer. + * @param ctx the channel handler context used for writing the response + * @param dualMemoryLimiter the memory limiter used to acquire and release memory permits + * @param isCancelled supplier that indicates if the permit acquisition should be cancelled + * @param command the base command to serialize and write to the channel + * @param permitAcquireErrorHandler handler for errors that occur during permit acquisition + * @return a future that completes when the command has been written to the channel's outbound buffer */ public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush(ChannelHandlerContext ctx, AsyncDualMemoryLimiter @@ -105,7 +118,6 @@ public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush dualMemoryLimiter.release(permits); throw e; } - }).thenAccept(__ -> { - }); + }).thenApply(__ -> null); } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java index 9a8cd86c57923..7fb8a17b45663 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java @@ -30,6 +30,9 @@ public interface AsyncSemaphore { * Returned future completes when permits are available. * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * + * @param permits number of permits to acquire + * @param isCancelled supplier that returns true if acquisition should be cancelled * @return CompletableFuture that completes with permit when available */ CompletableFuture acquire(long permits, BooleanSupplier isCancelled); @@ -39,16 +42,22 @@ public interface AsyncSemaphore { * Returns a future that completes when permits are available. * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * + * @param permit previously acquired permit to update + * @param newPermits new number of permits to update to + * @param isCancelled supplier that returns true if update should be cancelled * @return CompletableFuture that completes with permit when available */ CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, BooleanSupplier isCancelled); + /** * Release previously acquired permit. * Must be called to prevent permit leaks. + * + * @param permit permit to release */ void release(AsyncSemaphorePermit permit); - /** * Get the number of available permits. */ @@ -64,6 +73,9 @@ CompletableFuture update(AsyncSemaphorePermit permit, long */ int getQueueSize(); + /** + * Abstract base class for all exceptions thrown by acquire or update. + */ abstract class PermitAcquireException extends RuntimeException { public PermitAcquireException(String message) { super(message); @@ -88,12 +100,18 @@ public PermitAcquireQueueFullException(String message) { } } + /** + * Exception thrown when permit acquisition is attempted on a closed semaphore. + */ class PermitAcquireAlreadyClosedException extends PermitAcquireException { public PermitAcquireAlreadyClosedException(String message) { super(message); } } + /** + * Exception thrown when permit acquisition is cancelled. + */ class PermitAcquireCancelledException extends PermitAcquireException { public PermitAcquireCancelledException(String message) { super(message); @@ -101,7 +119,7 @@ public PermitAcquireCancelledException(String message) { } /** - * Represents a permit that can be updated or released. + * Represents an acquired permit that can be updated or released. */ interface AsyncSemaphorePermit { long getPermits(); From 5c1ff14dcaae92ea4742957e6f1af2419debea72 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 13 Oct 2025 22:40:53 +0300 Subject: [PATCH 19/75] Synchronize relevant details in pip-442.md --- pip/pip-442.md | 438 +++++++++++++++++++++++++++---------------------- 1 file changed, 239 insertions(+), 199 deletions(-) diff --git a/pip/pip-442.md b/pip/pip-442.md index d629572f61133..3c702ef955a5c 100644 --- a/pip/pip-442.md +++ b/pip/pip-442.md @@ -132,12 +132,11 @@ public interface AsyncSemaphore { /** * Acquire permits from the semaphore. * Returned future completes when permits are available. - * It will complete exceptionally with PermitAcquireTimeoutException on timeout, - * PermitAcquireQueueFullException when queue full, - * PermitAcquireCancelledException when cancelled, - * and PermitAcquireAlreadyClosedException when closed. - * @param permits number of permits to acquire - * @param isCancelled supplier to check if request should be cancelled + * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * + * @param permits number of permits to acquire + * @param isCancelled supplier that returns true if acquisition should be cancelled * @return CompletableFuture that completes with permit when available */ CompletableFuture acquire(long permits, BooleanSupplier isCancelled); @@ -145,20 +144,24 @@ public interface AsyncSemaphore { /** * Acquire or release permits for previously acquired permits by updating the permits. * Returns a future that completes when permits are available. - * @param permit existing permit to update - * @param newPermits new permit count - * @param isCancelled supplier to check if request should be cancelled + * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * + * @param permit previously acquired permit to update + * @param newPermits new number of permits to update to + * @param isCancelled supplier that returns true if update should be cancelled * @return CompletableFuture that completes with permit when available */ - CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, + CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, BooleanSupplier isCancelled); - + /** * Release previously acquired permit. - * Must be called to prevent memory permit leaks. + * Must be called to prevent permit leaks. + * + * @param permit permit to release */ void release(AsyncSemaphorePermit permit); - /** * Get the number of available permits. */ @@ -186,13 +189,16 @@ public interface AsyncDualMemoryLimiter { HEAP_MEMORY, // For heap memory allocation DIRECT_MEMORY // For direct memory allocation } - + /** * Acquire permits for the specified memory size. * Returned future completes when memory permits are available. - * @param memorySize size in bytes - * @param limitType type of memory limit - * @param isCancelled supplier to check if request should be cancelled + * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full + * + * @param memorySize the size of memory to acquire permits for + * @param limitType the type of memory limit (HEAP_MEMORY or DIRECT_MEMORY) + * @param isCancelled supplier that returns true if acquisition should be cancelled * @return CompletableFuture that completes with permit when available */ CompletableFuture acquire(long memorySize, LimitType limitType, @@ -200,46 +206,62 @@ public interface AsyncDualMemoryLimiter { /** * Acquire or release permits for previously acquired permits by updating the requested memory size. - * @param permit existing permit - * @param newMemorySize new size in bytes - * @param isCancelled supplier to check if request should be cancelled + * Returns a future that completes when permits are available. + * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full + * The provided permit is released when the permits are successfully acquired and the returned updated + * permit replaces the old instance. + * + * @param permit the previously acquired permit to update + * @param newMemorySize the new memory size to update to + * @param isCancelled supplier that returns true if update should be cancelled * @return CompletableFuture that completes with permit when available */ - CompletableFuture update(AsyncDualMemoryLimiterPermit permit, - long newMemorySize, + CompletableFuture update(AsyncDualMemoryLimiterPermit permit, long newMemorySize, BooleanSupplier isCancelled); - + /** * Release previously acquired permit. + * Must be called to prevent memory permit leaks. + * + * @param permit the permit to release */ void release(AsyncDualMemoryLimiterPermit permit); - /** - * Helper method to acquire permits, execute function, and release permits. - * @param memorySize size in bytes - * @param limitType type of memory limit - * @param isCancelled supplier to check if request should be cancelled + * Execute the specified function with acquired permits and release the permits after the returned future completes. + * @param memorySize memory size to acquire permits for + * @param limitType memory limit type to acquire permits for * @param function function to execute with acquired permits - * @param permitAcquireErrorHandler error handler for permit acquisition failures * @return result of the function + * @param type of the CompletableFuture returned by the function */ default CompletableFuture withAcquiredPermits(long memorySize, LimitType limitType, BooleanSupplier isCancelled, Function> function, Function> - permitAcquireErrorHandler); - + permitAcquireErrorHandler) { + return AsyncDualMemoryLimiterUtil.withPermitsFuture(acquire(memorySize, limitType, isCancelled), function, + permitAcquireErrorHandler, this::release); + } + /** - * Helper method to update permits, execute function, and release permits. + * Executed the specified function with updated permits and release the permits after the returned future completes. + * @param initialPermit initial permit to update + * @param newMemorySize new memory size to update to + * @param function function to execute with updated permits + * @return result of the function + * @param type of the CompletableFuture returned by the function */ - default CompletableFuture withUpdatedPermits(AsyncDualMemoryLimiterPermit initialPermit, - long newMemorySize, + default CompletableFuture withUpdatedPermits(AsyncDualMemoryLimiterPermit initialPermit, long newMemorySize, BooleanSupplier isCancelled, Function> function, Function> - permitAcquireErrorHandler); + permitAcquireErrorHandler) { + return AsyncDualMemoryLimiterUtil.withPermitsFuture(update(initialPermit, newMemorySize, isCancelled), function, + permitAcquireErrorHandler, this::release); + } } ``` @@ -250,15 +272,48 @@ A utility class provides helper methods for common patterns: ```java public class AsyncDualMemoryLimiterUtil { /** - * Acquire permits and write the command as the response to the channel. - * Releases the permits after the response has been written or the write fails. + * Execute a function with acquired permits and ensure permits are released after completion. + * This method handles the lifecycle of permits - acquisition, usage, and release, including error cases. + * + * @param permitsFuture Future that will complete with the required permits + * @param function Function to execute once permits are acquired that returns a CompletableFuture + * @param permitAcquireErrorHandler Handler for permit acquisition errors that returns a CompletableFuture + * @param releaser Consumer that handles releasing the permits + * @param The type of result returned by the function + * @return CompletableFuture that completes with the result of the function execution + */ + public static CompletableFuture withPermitsFuture( + CompletableFuture + permitsFuture, + Function> function, + Function> + permitAcquireErrorHandler, + Consumer releaser) { + // implementation omitted from PIP document + } + + /** + * Acquires permits and writes the command as a response to the channel. + * Releases the permits after the response has been written to the socket or if the write fails. + * + * @param ctx the channel handler context used for writing the response + * @param dualMemoryLimiter the memory limiter used to acquire and release memory permits + * @param isCancelled supplier that indicates if the permit acquisition should be cancelled + * @param command the base command to serialize and write to the channel + * @param permitAcquireErrorHandler handler for errors that occur during permit acquisition + * @return a future that completes when the command has been written to the channel's outbound buffer */ - public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush( - ChannelHandlerContext ctx, - AsyncDualMemoryLimiter dualMemoryLimiter, - BooleanSupplier isCancelled, - BaseCommand command, - Consumer permitAcquireErrorHandler); + public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush(ChannelHandlerContext ctx, + AsyncDualMemoryLimiter + dualMemoryLimiter, + BooleanSupplier isCancelled, + BaseCommand command, + Consumer + permitAcquireErrorHandler + ) { + // implementation omitted from PIP document + } } ``` @@ -269,68 +324,75 @@ public class AsyncDualMemoryLimiterUtil { In `ServerCnx.handleGetTopicsOfNamespace`, the implementation uses the helper methods: ```java -private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName namespaceName, - long requestId, CommandGetTopicsOfNamespace.Mode mode, - Optional topicsPattern, Optional topicsHash, - Semaphore lookupSemaphore) { + private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName namespaceName, long requestId, + CommandGetTopicsOfNamespace.Mode mode, + Optional topicsPattern, Optional topicsHash, + Semaphore lookupSemaphore) { BooleanSupplier isPermitRequestCancelled = () -> !ctx().channel().isActive(); - - // Acquire initial heap permit with 1KB estimate - maxTopicListInFlightLimiter.withAcquiredPermits( - INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, // 1KB - AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, - isPermitRequestCancelled, - initialPermits -> { - return getBrokerService().pulsar().getNamespaceService() - .getListOfUserTopics(namespaceName, mode) - .thenAccept(topics -> { - // Calculate actual size - long actualSize = topics.stream().mapToInt(String::length).sum(); - - // Update to actual size - maxTopicListInFlightLimiter.withUpdatedPermits( - initialPermits, actualSize, isPermitRequestCancelled, - permits -> { - // Filter and process topics - List filteredTopics = filterTopics(topics, topicsPattern); - String hash = TopicList.calculateHash(filteredTopics); - boolean hashUnchanged = topicsHash.isPresent() - && topicsHash.get().equals(hash); - - // Send response with direct memory limiting - commandSender.sendGetTopicsOfNamespaceResponse( - filteredTopics, hash, filterTopics, - !hashUnchanged, requestId, - ex -> { - // Handle direct memory permit acquisition failure - log.warn("[{}] Failed to acquire direct memory permits", - remoteAddress, ex); - commandSender.sendErrorResponse(requestId, - ServerError.TooManyRequests, - "Cannot acquire permits for direct memory"); + maxTopicListInFlightLimiter.withAcquiredPermits(INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { + return getBrokerService().pulsar().getNamespaceService().getListOfUserTopics(namespaceName, mode) + .thenAccept(topics -> { + long actualSize = topics.stream().mapToInt(String::length).sum(); + maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, + isPermitRequestCancelled, permits -> { + boolean filterTopics = false; + // filter system topic + List filteredTopics = topics; + + if (enableSubscriptionPatternEvaluation && topicsPattern.isPresent()) { + if (topicsPattern.get().length() <= maxSubscriptionPatternLength) { + filterTopics = true; + filteredTopics = TopicList.filterTopics(filteredTopics, topicsPattern.get(), + topicsPatternImplementation); + } else { + log.info("[{}] Subscription pattern provided [{}] was longer than " + + "maximum {}.", remoteAddress, topicsPattern.get(), + maxSubscriptionPatternLength); + } + } + String hash = TopicList.calculateHash(filteredTopics); + boolean hashUnchanged = topicsHash.isPresent() && topicsHash.get().equals(hash); + if (hashUnchanged) { + filteredTopics = Collections.emptyList(); + } + if (log.isDebugEnabled()) { + log.debug("[{}] Received CommandGetTopicsOfNamespace for namespace " + + "[//{}] by {}, size:{}", remoteAddress, namespace, + requestId, + topics.size()); + } + commandSender.sendGetTopicsOfNamespaceResponse(filteredTopics, hash, filterTopics, + !hashUnchanged, requestId, ex -> { + log.warn("[{}] Failed to acquire direct memory permits for " + + "GetTopicsOfNamespace: {}", remoteAddress, ex.getMessage()); + commandSender.sendErrorResponse(requestId, ServerError.TooManyRequests, + "Cannot acquire permits for direct memory"); + }); + return CompletableFuture.completedFuture(null); + }, t -> { + log.warn("[{}] Failed to acquire heap memory permits for " + + "GetTopicsOfNamespace: {}", remoteAddress, t.getMessage()); + writeAndFlush(Commands.newError(requestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + return CompletableFuture.completedFuture(null); }); - return CompletableFuture.completedFuture(null); - }, - t -> { - // Handle heap memory update failure - log.warn("[{}] Failed to acquire heap memory permits", - remoteAddress, t); - writeAndFlush(Commands.newError(requestId, - ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); - return CompletableFuture.completedFuture(null); - }); - }).whenComplete((__, ___) -> { - lookupSemaphore.release(); - }); - }, - t -> { - // Handle initial heap memory acquisition failure - log.warn("[{}] Failed to acquire initial heap memory permits", - remoteAddress, t); - writeAndFlush(Commands.newError(requestId, - ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); + }).whenComplete((__, ___) -> { + lookupSemaphore.release(); + }).exceptionally(ex -> { + log.warn("[{}] Error GetTopicsOfNamespace for namespace [//{}] by {}", remoteAddress, + namespace, requestId); + commandSender.sendErrorResponse(requestId, + BrokerServiceException.getClientErrorCode(new ServerMetadataException(ex)), + ex.getMessage()); + return null; + }); + }, t -> { + log.warn("[{}] Failed to acquire initial heap memory permits for GetTopicsOfNamespace: {}", + remoteAddress, t.getMessage()); + writeAndFlush(Commands.newError(requestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + lookupSemaphore.release(); return CompletableFuture.completedFuture(null); }); } @@ -345,55 +407,48 @@ Modified `PulsarCommandSenderImpl`: public void sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, boolean filtered, boolean changed, long requestId, Consumer permitAcquireErrorHandler) { - BaseCommand command = Commands.newGetTopicsOfNamespaceResponseCommand( - topics, topicsHash, filtered, changed, requestId); + BaseCommand command = Commands.newGetTopicsOfNamespaceResponseCommand(topics, topicsHash, + filtered, changed, requestId); safeIntercept(command, cnx); - - // Use utility method to acquire permits, serialize, write, and release - acquireDirectMemoryPermitsAndWriteAndFlush( - cnx.ctx(), maxTopicListInFlightLimiter, - () -> !cnx.isActive(), - command, permitAcquireErrorHandler); + acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), + command, permitAcquireErrorHandler); } ``` The utility method implementation: ```java -public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush( - ChannelHandlerContext ctx, - AsyncDualMemoryLimiter dualMemoryLimiter, - BooleanSupplier isCancelled, - BaseCommand command, - Consumer permitAcquireErrorHandler) { - +public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush(ChannelHandlerContext ctx, + AsyncDualMemoryLimiter + dualMemoryLimiter, + BooleanSupplier isCancelled, + BaseCommand command, + Consumer + permitAcquireErrorHandler +) { // Calculate serialized size before acquiring permits int serializedSize = command.getSerializedSize(); - // Acquire permits - return dualMemoryLimiter.acquire(serializedSize, - AsyncDualMemoryLimiter.LimitType.DIRECT_MEMORY, - isCancelled) - .whenComplete((permits, t) -> { - if (t != null) { - permitAcquireErrorHandler.accept(t); - return; - } - try { - // Serialize with pre-calculated size - ByteBuf outBuf = Commands.serializeWithPrecalculatedSerializedSize( - command, serializedSize); - - // Write and release permits after completion - ctx.writeAndFlush(outBuf).addListener(future -> { + return dualMemoryLimiter.acquire(serializedSize, AsyncDualMemoryLimiter.LimitType.DIRECT_MEMORY, isCancelled) + .whenComplete((permits, t) -> { + if (t != null) { + permitAcquireErrorHandler.accept(t); + return; + } + try { + // Serialize the response + ByteBuf outBuf = Commands.serializeWithPrecalculatedSerializedSize(command, serializedSize); + // Write the response + ctx.writeAndFlush(outBuf).addListener(future -> { + // Release permits after the response has been written to the socket + dualMemoryLimiter.release(permits); + }); + } catch (Exception e) { + // Return permits if an exception occurs before writeAndFlush is called successfully dualMemoryLimiter.release(permits); - }); - } catch (Exception e) { - // Release permits if exception occurs before writeAndFlush - dualMemoryLimiter.release(permits); - throw e; - } - }).thenAccept(__ -> {}); + throw e; + } + }).thenApply(__ -> null); } ``` @@ -418,72 +473,57 @@ public void sendTopicListUpdate(long watcherId, String topicsHash, On the Pulsar Proxy side in `LookupProxyHandler`: ```java -private void internalPerformGetTopicsOfNamespace(long clientRequestId, String namespaceName, - ClientCnx clientCnx, ByteBuf command, - long requestId) { +private void internalPerformGetTopicsOfNamespace(long clientRequestId, String namespaceName, ClientCnx clientCnx, + ByteBuf command, long requestId) { BooleanSupplier isPermitRequestCancelled = () -> !proxyConnection.ctx().channel().isActive(); - - // Acquire initial heap permit - maxTopicListInFlightLimiter.withAcquiredPermits( - INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, - AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, - isPermitRequestCancelled, - initialPermits -> { - return clientCnx.newGetTopicsOfNamespace(command, requestId) - .whenComplete((r, t) -> { + maxTopicListInFlightLimiter.withAcquiredPermits(INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { + return clientCnx.newGetTopicsOfNamespace(command, requestId).whenComplete((r, t) -> { if (t != null) { - log.warn("[{}] Failed to get TopicsOfNamespace {}", - clientAddress, namespaceName, t); - writeAndFlush(Commands.newError(clientRequestId, - getServerError(t), t.getMessage())); + log.warn("[{}] Failed to get TopicsOfNamespace {}: {}", clientAddress, namespaceName, + t.getMessage()); + writeAndFlush(Commands.newError(clientRequestId, getServerError(t), t.getMessage())); } else { - // Update to actual size - long actualSize = r.getNonPartitionedOrPartitionTopics() - .stream().mapToInt(String::length).sum(); - - maxTopicListInFlightLimiter.withUpdatedPermits( - initialPermits, actualSize, isPermitRequestCancelled, - permits -> { - return handleWritingGetTopicsResponse( - clientRequestId, r, isPermitRequestCancelled); - }, - t2 -> { - log.warn("[{}] Failed to acquire heap memory permits", - clientAddress, t2); - writeAndFlush(Commands.newError(clientRequestId, - ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); - return CompletableFuture.completedFuture(null); - }); + long actualSize = + r.getNonPartitionedOrPartitionTopics().stream().mapToInt(String::length).sum(); + maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, + isPermitRequestCancelled, permits -> { + return handleWritingGetTopicsResponse(clientRequestId, r, isPermitRequestCancelled); + }, t2 -> { + log.warn("[{}] Failed to acquire actual heap memory permits for " + + "GetTopicsOfNamespace: {}", clientAddress, t2.getMessage()); + writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + + return CompletableFuture.completedFuture(null); + }); } }).thenApply(__ -> null); - }, - t -> { - log.warn("[{}] Failed to acquire initial heap memory permits", - clientAddress, t); - writeAndFlush(Commands.newError(clientRequestId, - ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); - return CompletableFuture.completedFuture(null); - }); + }, t -> { + log.warn("[{}] Failed to acquire initial heap memory permits for GetTopicsOfNamespace: {}", + clientAddress, t.getMessage()); + writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + + return CompletableFuture.completedFuture(null); + }).exceptionally(ex -> { + writeAndFlush(Commands.newError(clientRequestId, getServerError(ex), ex.getMessage())); + return null; + }); } -private CompletableFuture handleWritingGetTopicsResponse( - long clientRequestId, GetTopicsResult r, BooleanSupplier isCancelled) { +private CompletableFuture handleWritingGetTopicsResponse(long clientRequestId, GetTopicsResult r, + BooleanSupplier isCancelled) { BaseCommand responseCommand = Commands.newGetTopicsOfNamespaceResponseCommand( - r.getNonPartitionedOrPartitionTopics(), r.getTopicsHash(), - r.isFiltered(), r.isChanged(), clientRequestId); - - return acquireDirectMemoryPermitsAndWriteAndFlush( - proxyConnection.ctx(), maxTopicListInFlightLimiter, - isCancelled, responseCommand, - t -> { - log.warn("[{}] Failed to acquire direct memory permits", - clientAddress, t); - writeAndFlush(Commands.newError(clientRequestId, - ServerError.TooManyRequests, - "Failed due to direct memory limit exceeded")); - }); + r.getNonPartitionedOrPartitionTopics(), r.getTopicsHash(), r.isFiltered(), + r.isChanged(), clientRequestId); + return acquireDirectMemoryPermitsAndWriteAndFlush(proxyConnection.ctx(), maxTopicListInFlightLimiter, + isCancelled, responseCommand, t -> { + log.warn("[{}] Failed to acquire actual direct memory permits for GetTopicsOfNamespace: {}", + clientAddress, t.getMessage()); + writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + }); } ``` From a754edbba5b782bb7ed5a30e1317c836eb55cb96 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Oct 2025 16:36:22 +0300 Subject: [PATCH 20/75] Add retries to watch topic list success and update --- .../broker/service/PulsarCommandSender.java | 8 +- .../service/PulsarCommandSenderImpl.java | 18 ++- .../broker/service/TopicListService.java | 89 +++++++++-- .../broker/service/TopicListServiceTest.java | 142 +++++++++++++++++- 4 files changed, 228 insertions(+), 29 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java index 1f670ddee3bac..e0ef1b9011681 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java @@ -22,6 +22,7 @@ import io.netty.util.concurrent.Future; import java.util.List; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; import org.apache.bookkeeper.mledger.Entry; import org.apache.pulsar.client.api.transaction.TxnID; @@ -94,10 +95,11 @@ Future sendMessagesToConsumer(long consumerId, String topicName, Subscript void sendEndTxnErrorResponse(long requestId, TxnID txnID, ServerError error, String message); - void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics, - Consumer permitAcquireErrorHandler); + CompletableFuture sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, + List topics, + Consumer permitAcquireErrorHandler); - void sendWatchTopicListUpdate(long watcherId, + CompletableFuture sendWatchTopicListUpdate(long watcherId, List newTopics, List deletedTopics, String topicsHash, Consumer permitAcquireErrorHandler); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java index 730241e3118a5..6d619123a213a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; @@ -365,26 +366,29 @@ public void sendEndTxnErrorResponse(long requestId, TxnID txnID, ServerError err /*** * @param topics topic names which are matching, the topic name contains the partition suffix. + * @return */ @Override - public void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics, - Consumer permitAcquireErrorHandler) { + public CompletableFuture sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, + List topics, + Consumer permitAcquireErrorHandler) { BaseCommand command = Commands.newWatchTopicListSuccess(requestId, watcherId, topicsHash, topics); safeIntercept(command, cnx); - acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), + return acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), command, permitAcquireErrorHandler); } /*** * {@inheritDoc} + * @return */ @Override - public void sendWatchTopicListUpdate(long watcherId, - List newTopics, List deletedTopics, String topicsHash, - Consumer permitAcquireErrorHandler) { + public CompletableFuture sendWatchTopicListUpdate(long watcherId, List newTopics, + List deletedTopics, String topicsHash, + Consumer permitAcquireErrorHandler) { BaseCommand command = Commands.newWatchTopicUpdate(watcherId, newTopics, deletedTopics, topicsHash); safeIntercept(command, cnx); - acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), + return acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), command, permitAcquireErrorHandler); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index ad633a68ab897..d50311edccf88 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -22,8 +22,15 @@ import java.util.HashSet; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.TopicResources; @@ -31,16 +38,20 @@ import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.semaphore.AsyncSemaphore; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.topics.TopicsPattern; import org.apache.pulsar.common.topics.TopicsPatternFactory; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.Runnables; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; import org.apache.pulsar.metadata.api.NotificationType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TopicListService { - + private static final int MAX_RETRY_COUNT = 10; public static class TopicListWatcher implements BiConsumer { @@ -101,6 +112,7 @@ public void accept(String topicName, NotificationType notificationType) { private final boolean enableSubscriptionPatternEvaluation; private final int maxSubscriptionPatternLength; private final ConcurrentLongHashMap> watchers; + private final Backoff retryBackoff; public TopicListService(PulsarService pulsar, ServerCnx connection, @@ -114,6 +126,10 @@ public TopicListService(PulsarService pulsar, ServerCnx connection, .concurrencyLevel(1) .build(); this.topicResources = pulsar.getPulsarResources().getTopicResources(); + this.retryBackoff = new Backoff( + 100, TimeUnit.MILLISECONDS, + 25, TimeUnit.SECONDS, + 0, TimeUnit.MILLISECONDS); } public void inactivate() { @@ -203,13 +219,7 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo "[{}] Received WatchTopicList for namespace [//{}] by {}", connection.toString(), namespaceName, requestId); } - connection.getCommandSender().sendWatchTopicListSuccess(requestId, watcherId, hash, topicList, - t -> { - // TODO add retry with backoff - log.warn("[{}] Cannot acquire direct memory tokens for sending topic list success." - + "State will be inconsistent on the client. {}", - connection.toString(), t.getMessage()); - }); + sendTopicListSuccessWithRetries(watcherId, requestId, topicList, hash); lookupSemaphore.release(); }) .exceptionally(ex -> { @@ -224,6 +234,12 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo }); } + private void sendTopicListSuccessWithRetries(long watcherId, long requestId, List topicList, String hash) { + performOperationWithRetries("topic list success", permitAcquireErrorHandler -> + () -> connection.getCommandSender() + .sendWatchTopicListSuccess(requestId, watcherId, hash, topicList, permitAcquireErrorHandler)); + } + /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). */ @@ -294,13 +310,54 @@ public void deleteTopicListWatcher(Long watcherId) { */ public void sendTopicListUpdate(long watcherId, String topicsHash, List deletedTopics, List newTopics) { - connection.getCommandSender().sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash, - t -> { - // TODO add retry with backoff - log.warn( - "[{}] Cannot acquire direct memory tokens for sending topic list update. State will be " - + "inconsistent on the client. {}", - connection.toString(), t.getMessage()); - }); + performOperationWithRetries("topic list update", permitAcquireErrorHandler -> + () -> connection.getCommandSender() + .sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash, permitAcquireErrorHandler)); + } + + // performs an operation with retries, if the operation fails, it will retry after a backoff period + private void performOperationWithRetries(String operationName, + Function, Supplier>> + asyncOperationFactory) { + // holds a reference to the operation, this is to resolve a circular dependency between the error handler and + // the actual operation + AtomicReference operationRef = new AtomicReference<>(); + // create the error handler for the operation + Consumer permitAcquireErrorHandler = + createPermitAcquireErrorHandler(operationName, operationRef); + // create the async operation using the factory function. Pass the error handler to the factory function. + Supplier> asyncOperation = asyncOperationFactory.apply(permitAcquireErrorHandler); + // set the operation to run into the operation reference + operationRef.set(Runnables.catchingAndLoggingThrowables(() -> { + asyncOperation.get().thenRun(() -> retryBackoff.reset()); + })); + // run the operation + operationRef.get().run(); + } + + // retries an operation up to MAX_RETRY_COUNT times with backoff + private Consumer createPermitAcquireErrorHandler(String operationName, + AtomicReference operationRef) { + ScheduledExecutorService scheduledExecutor = connection.ctx().channel().eventLoop(); + AtomicInteger retryCount = new AtomicInteger(0); + return t -> { + Throwable unwrappedException = FutureUtil.unwrapCompletionException(t); + if (unwrappedException instanceof AsyncSemaphore.PermitAcquireCancelledException + || unwrappedException instanceof AsyncSemaphore.PermitAcquireAlreadyClosedException + || !connection.isActive()) { + return; + } + if (retryCount.incrementAndGet() < MAX_RETRY_COUNT) { + long retryDelay = retryBackoff.next(); + log.info("[{}] Cannot acquire direct memory tokens for sending {}. Retry {}/{} in {} ms. {}", + connection, operationName, retryCount.get(), MAX_RETRY_COUNT, retryDelay, + t.getMessage()); + scheduledExecutor.schedule(operationRef.get(), retryDelay, TimeUnit.MILLISECONDS); + } else { + log.warn("[{}] Cannot acquire direct memory tokens for sending {}." + + "State will be inconsistent on the client. {}", connection, operationName, + t.getMessage()); + } + }; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java index ba042470c0e3b..3c12f03d2406b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java @@ -18,17 +18,30 @@ */ package org.apache.pulsar.broker.service; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.ScheduledFuture; import java.net.InetSocketAddress; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import org.apache.commons.collections4.ListUtils; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceService; @@ -37,8 +50,12 @@ import org.apache.pulsar.common.api.proto.CommandWatchTopicListClose; import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.semaphore.AsyncSemaphore; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.topics.TopicsPattern; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.Notification; +import org.apache.pulsar.metadata.api.NotificationType; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -52,13 +69,24 @@ public class TopicListServiceTest { private TopicResources topicResources; private final TopicsPattern.RegexImplementation topicsPatternImplementation = TopicsPattern.RegexImplementation.RE2J_WITH_JDK_FALLBACK; + private EventLoop eventLoop; + private PulsarCommandSender pulsarCommandSender; + private Consumer notificationConsumer; @BeforeMethod(alwaysRun = true) public void setup() throws Exception { lookupSemaphore = new Semaphore(1); lookupSemaphore.acquire(); topicListFuture = new CompletableFuture<>(); - topicResources = mock(TopicResources.class); + + AtomicReference> listenerRef = new AtomicReference<>(); + MetadataStore metadataStore = mock(MetadataStore.class); + doAnswer(invocationOnMock -> { + listenerRef.set(invocationOnMock.getArgument(0)); + return null; + }).when(metadataStore).registerListener(any()); + topicResources = spy(new TopicResources(metadataStore)); + notificationConsumer = listenerRef.get(); PulsarService pulsar = mock(PulsarService.class); when(pulsar.getNamespaceService()).thenReturn(mock(NamespaceService.class)); @@ -66,10 +94,23 @@ public void setup() throws Exception { when(pulsar.getPulsarResources().getTopicResources()).thenReturn(topicResources); when(pulsar.getNamespaceService().getListOfPersistentTopics(any())).thenReturn(topicListFuture); - connection = mock(ServerCnx.class); when(connection.getRemoteAddress()).thenReturn(new InetSocketAddress(10000)); - when(connection.getCommandSender()).thenReturn(mock(PulsarCommandSender.class)); + pulsarCommandSender = mock(PulsarCommandSender.class); + when(connection.getCommandSender()).thenReturn(pulsarCommandSender); + when(connection.isActive()).thenReturn(true); + when(pulsarCommandSender.sendWatchTopicListUpdate(anyLong(), any(), any(), anyString(), any())) + .thenReturn(CompletableFuture.completedFuture(null)); + when(pulsarCommandSender.sendWatchTopicListSuccess(anyLong(), anyLong(), anyString(), any(), any())) + .thenReturn(CompletableFuture.completedFuture(null)); + + + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(connection.ctx()).thenReturn(ctx); + Channel channel = mock(Channel.class); + when(ctx.channel()).thenReturn(channel); + eventLoop = mock(EventLoop.class); + when(channel.eventLoop()).thenReturn(eventLoop); topicListService = new TopicListService(pulsar, connection, true, 30); @@ -129,4 +170,99 @@ public void testCommandWatchTopicListCloseRemovesListener() { verify(topicResources).deregisterPersistentTopicListener(any(TopicListService.TopicListWatcher.class)); } + @Test + public void testCommandWatchSuccessRetries() { + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + List topics = Collections.singletonList("persistent://tenant/ns/topic1"); + String hash = TopicList.calculateHash(topics); + doAnswer(invocationOnMock -> { + Runnable runnable = invocationOnMock.getArgument(0); + // run immediately + runnable.run(); + return mock(ScheduledFuture.class); + }).when(eventLoop).schedule(any(Runnable.class), anyLong(), any()); + AtomicInteger failureCount = new AtomicInteger(0); + doAnswer(invocationOnMock -> { + if (failureCount.incrementAndGet() < 3) { + Throwable failure = new AsyncSemaphore.PermitAcquireTimeoutException("Acquire timed out"); + Consumer permitAcquireErrorHandler = invocationOnMock.getArgument(4); + permitAcquireErrorHandler.accept(failure); + return CompletableFuture.failedFuture(failure); + } else { + return CompletableFuture.completedFuture(null); + } + }).when(pulsarCommandSender).sendWatchTopicListSuccess(anyLong(), anyLong(), anyString(), any(), any()); + topicListFuture.complete(topics); + verify(connection.getCommandSender(), times(3)) + .sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(topics), any()); + } + + @Test + public void testCommandWatchUpdate() { + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + List topics = Collections.singletonList("persistent://tenant/ns/topic1"); + topicListFuture.complete(topics); + + List newTopics = Collections.singletonList("persistent://tenant/ns/topic2"); + String hash = TopicList.calculateHash(ListUtils.union(topics, newTopics)); + notificationConsumer.accept( + new Notification(NotificationType.Created, "/managed-ledgers/tenant/ns/persistent/topic2")); + verify(connection.getCommandSender()) + .sendWatchTopicListUpdate(eq(13L), eq(newTopics), any(), eq(hash), any()); + + hash = TopicList.calculateHash(newTopics); + notificationConsumer.accept( + new Notification(NotificationType.Deleted, "/managed-ledgers/tenant/ns/persistent/topic1")); + verify(connection.getCommandSender()) + .sendWatchTopicListUpdate(eq(13L), eq(List.of()), eq(topics), eq(hash), any()); + } + + @Test + public void testCommandWatchUpdateRetries() { + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + List topics = Collections.singletonList("persistent://tenant/ns/topic1"); + topicListFuture.complete(topics); + + List newTopics = Collections.singletonList("persistent://tenant/ns/topic2"); + String hash = TopicList.calculateHash(ListUtils.union(topics, newTopics)); + doAnswer(invocationOnMock -> { + Runnable runnable = invocationOnMock.getArgument(0); + // run immediately + runnable.run(); + return mock(ScheduledFuture.class); + }).when(eventLoop).schedule(any(Runnable.class), anyLong(), any()); + AtomicInteger failureCount = new AtomicInteger(0); + doAnswer(invocationOnMock -> { + if (failureCount.incrementAndGet() < 3) { + Throwable failure = new AsyncSemaphore.PermitAcquireTimeoutException("Acquire timed out"); + Consumer permitAcquireErrorHandler = invocationOnMock.getArgument(4); + permitAcquireErrorHandler.accept(failure); + return CompletableFuture.failedFuture(failure); + } else { + return CompletableFuture.completedFuture(null); + } + }).when(pulsarCommandSender).sendWatchTopicListUpdate(anyLong(), any(), any(), anyString(), any()); + notificationConsumer.accept( + new Notification(NotificationType.Created, "/managed-ledgers/tenant/ns/persistent/topic2")); + verify(connection.getCommandSender(), times(3)) + .sendWatchTopicListUpdate(eq(13L), eq(newTopics), eq(List.of()), eq(hash), any()); + } } From e3c59de56330645d770ed420b6ce34c065808029 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Oct 2025 19:11:27 +0300 Subject: [PATCH 21/75] Replace ConcurrentLinkedQueue with ArrayBlockingQueue --- .../common/semaphore/AsyncSemaphoreImpl.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index b3ae4fd6b081b..1543e0654ee8e 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -19,8 +19,9 @@ package org.apache.pulsar.common.semaphore; import io.netty.util.concurrent.DefaultThreadFactory; +import java.util.Queue; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -41,9 +42,8 @@ public class AsyncSemaphoreImpl implements AsyncSemaphore, AutoCloseable { private static final Logger log = LoggerFactory.getLogger(AsyncSemaphoreImpl.class); private final AtomicLong availablePermits; - private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + private final Queue queue; private final long maxPermits; - private final int maxQueueSize; private final long timeoutMillis; private final ScheduledExecutorService executor; private final boolean shutdownExecutor; @@ -64,7 +64,7 @@ public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis, boolean shutdownExecutor, LongConsumer queueLatencyRecorder) { this.availablePermits = new AtomicLong(maxPermits); this.maxPermits = maxPermits; - this.maxQueueSize = maxQueueSize; + this.queue = new ArrayBlockingQueue<>(maxQueueSize); this.timeoutMillis = timeoutMillis; this.executor = executor; this.shutdownExecutor = shutdownExecutor; @@ -94,14 +94,12 @@ private CompletableFuture internalAcquire(long permits, lo return future; } - if (queue.size() >= maxQueueSize) { + PendingRequest request = new PendingRequest(permits, acquirePermits, future, isCancelled); + if (!queue.offer(request)) { future.completeExceptionally(new PermitAcquireQueueFullException( "Semaphore queue is full")); return future; } - - PendingRequest request = new PendingRequest(permits, acquirePermits, future, isCancelled); - queue.add(request); // Schedule timeout ScheduledFuture timeoutTask = executor.schedule(() -> { if (!request.future.isDone() && queue.remove(request)) { From b40a73d2c298d78a1d1bbfd80081f5aab00f7719 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Oct 2025 19:16:22 +0300 Subject: [PATCH 22/75] Fix exception class name typos in javadocs --- pip/pip-442.md | 8 ++++---- .../apache/pulsar/common/semaphore/AsyncSemaphore.java | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/pip/pip-442.md b/pip/pip-442.md index 3c702ef955a5c..8261445ff5769 100644 --- a/pip/pip-442.md +++ b/pip/pip-442.md @@ -132,8 +132,8 @@ public interface AsyncSemaphore { /** * Acquire permits from the semaphore. * Returned future completes when permits are available. - * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout - * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full * * @param permits number of permits to acquire * @param isCancelled supplier that returns true if acquisition should be cancelled @@ -144,8 +144,8 @@ public interface AsyncSemaphore { /** * Acquire or release permits for previously acquired permits by updating the permits. * Returns a future that completes when permits are available. - * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout - * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full * * @param permit previously acquired permit to update * @param newPermits new number of permits to update to diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java index 7fb8a17b45663..0d648d5067260 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java @@ -28,8 +28,8 @@ public interface AsyncSemaphore { /** * Acquire permits from the semaphore. * Returned future completes when permits are available. - * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout - * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full * * @param permits number of permits to acquire * @param isCancelled supplier that returns true if acquisition should be cancelled @@ -40,8 +40,8 @@ public interface AsyncSemaphore { /** * Acquire or release permits for previously acquired permits by updating the permits. * Returns a future that completes when permits are available. - * It will complete exceptionally with AsyncSemaphorePermitAcquireTimeoutException on timeout - * and exceptionally with AsyncSemaphorePermitAcquireQueueFullException when queue full + * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout + * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full * * @param permit previously acquired permit to update * @param newPermits new number of permits to update to From fec104a6dc3a4a05382e2474c7bc46ffbecf8b54 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Oct 2025 19:18:36 +0300 Subject: [PATCH 23/75] Address review feedback --- pip/pip-442.md | 2 +- .../pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pip/pip-442.md b/pip/pip-442.md index 8261445ff5769..2748866ae8d12 100644 --- a/pip/pip-442.md +++ b/pip/pip-442.md @@ -443,7 +443,7 @@ public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush // Release permits after the response has been written to the socket dualMemoryLimiter.release(permits); }); - } catch (Exception e) { + } catch (Throwable e) { // Return permits if an exception occurs before writeAndFlush is called successfully dualMemoryLimiter.release(permits); throw e; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java index af9870437f4d2..616681dc15570 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtil.java @@ -113,7 +113,7 @@ public static CompletableFuture acquireDirectMemoryPermitsAndWriteAndFlush // Release permits after the response has been written to the socket dualMemoryLimiter.release(permits); }); - } catch (Exception e) { + } catch (Throwable e) { // Return permits if an exception occurs before writeAndFlush is called successfully dualMemoryLimiter.release(permits); throw e; From db594d935608764fa6a95b64a15a34966398e7d6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Oct 2025 19:20:32 +0300 Subject: [PATCH 24/75] Address review feedback --- .../java/org/apache/pulsar/proxy/server/LookupProxyHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java index 7c59f4fbc3040..86901820d0e4e 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java @@ -406,7 +406,7 @@ private CompletableFuture handleWritingGetTopicsResponse(long clientReques log.warn("[{}] Failed to acquire actual direct memory permits for GetTopicsOfNamespace: {}", clientAddress, t.getMessage()); writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); + "Failed due to direct memory limit exceeded")); }); } From a77f6da2fd532dd835ab98b4bf82ed704245029e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Oct 2025 19:23:31 +0300 Subject: [PATCH 25/75] Fix otel metric names for wait time --- .../pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java index 7b2d78b2c935d..ea0c6bad3b31d 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java @@ -193,7 +193,7 @@ public double get() { .quantile(0.99, 0.01) .quantile(1, 0.01) .create()); - this.otelHeapWaitTime = openTelemetryMeter.histogramBuilder("topic.list.heap.wait.time.ms") + this.otelHeapWaitTime = openTelemetryMeter.histogramBuilder("topic.list.heap.wait.time") .setUnit("s") .setDescription("Wait time for heap memory permits") .build(); @@ -205,7 +205,7 @@ public double get() { .quantile(0.99, 0.01) .quantile(1, 0.01) .create()); - this.otelDirectWaitTime = openTelemetryMeter.histogramBuilder("topic.list.direct.wait.time.ms") + this.otelDirectWaitTime = openTelemetryMeter.histogramBuilder("topic.list.direct.wait.time") .setUnit("s") .setDescription("Wait time for direct memory permits") .build(); From 61549838e27c567d9697fc21124f235ffe63e7c9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Oct 2025 19:36:43 +0300 Subject: [PATCH 26/75] Use closer approximation of size by using ByteBufUtil.utf8Bytes to calculate size --- .../java/org/apache/pulsar/broker/service/ServerCnx.java | 6 +++++- .../org/apache/pulsar/proxy/server/LookupProxyHandler.java | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 355f4b1b6614d..eeee6a36c71bb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -34,6 +34,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufUtil; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelOption; @@ -2586,7 +2587,10 @@ private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { return getBrokerService().pulsar().getNamespaceService().getListOfUserTopics(namespaceName, mode) .thenAccept(topics -> { - long actualSize = topics.stream().mapToInt(String::length).sum(); + long actualSize = topics.stream() + .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes + .map(n -> n + 32) // add 32 bytes overhead for each entry + .sum(); maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, isPermitRequestCancelled, permits -> { boolean filterTopics = false; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java index 86901820d0e4e..2fe3fd2d923a1 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java @@ -20,6 +20,7 @@ import static org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufUtil; import io.netty.channel.ChannelHandlerContext; import io.prometheus.client.Counter; import java.net.InetSocketAddress; @@ -369,7 +370,10 @@ private void internalPerformGetTopicsOfNamespace(long clientRequestId, String na writeAndFlush(Commands.newError(clientRequestId, getServerError(t), t.getMessage())); } else { long actualSize = - r.getNonPartitionedOrPartitionTopics().stream().mapToInt(String::length).sum(); + r.getNonPartitionedOrPartitionTopics().stream() + .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes + .map(n -> n + 32) // add 32 bytes overhead for each entry + .sum(); maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, isPermitRequestCancelled, permits -> { return handleWritingGetTopicsResponse(clientRequestId, r, isPermitRequestCancelled); From fbee63bd8a42b609b6353563b8cd005fc8b06f0d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Oct 2025 19:50:41 +0300 Subject: [PATCH 27/75] Address review feedback --- .../api/PatternConsumerBackPressureMultipleConsumersTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java index fb2e302b15966..2acbf38fe37e7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java @@ -144,6 +144,8 @@ public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() }); } catch (Exception e) { semaphore.release(); + latch.countDown(); + log.error("Failed to execute getTopicsUnderNamespace request.", e); } }); } From 3f7fffab57d561b35cf2b92d68afbcc4575798ed Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 15 Oct 2025 20:19:44 +0300 Subject: [PATCH 28/75] Address review feedback --- pip/pip-442.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pip/pip-442.md b/pip/pip-442.md index 2748866ae8d12..59a0cdebc04d5 100644 --- a/pip/pip-442.md +++ b/pip/pip-442.md @@ -522,7 +522,7 @@ private CompletableFuture handleWritingGetTopicsResponse(long clientReques log.warn("[{}] Failed to acquire actual direct memory permits for GetTopicsOfNamespace: {}", clientAddress, t.getMessage()); writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); + "Failed due to direct memory limit exceeded")); }); } ``` From 654fe7d9525be91907636e31551c7d24adb33522 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 15 Oct 2025 20:22:09 +0300 Subject: [PATCH 29/75] Address review feedback about race condition in internalProcessQueue --- .../common/semaphore/AsyncSemaphoreImpl.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index 1543e0654ee8e..5e20a07f29b8e 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -51,10 +51,24 @@ public class AsyncSemaphoreImpl implements AsyncSemaphore, AutoCloseable { private final AtomicBoolean closed = new AtomicBoolean(false); private final Runnable processQueueRunnable = Runnables.catchingAndLoggingThrowables(this::internalProcessQueue); + /** + * Creates an AsyncSemaphoreImpl with the given parameters. + * @param maxPermits max number of permits available for acquisition + * @param maxQueueSize max number of requests that can be queued + * @param timeoutMillis timeout in milliseconds for acquiring permits + */ public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis) { this(maxPermits, maxQueueSize, timeoutMillis, createExecutor(), true, null); } + /** + * Creates an AsyncSemaphoreImpl with the given parameters. + * @param maxPermits max number of permits available for acquisition + * @param maxQueueSize max number of requests that can be queued + * @param timeoutMillis timeout in milliseconds for acquiring permits + * @param executor executor service to use for scheduling timeouts, it is expected to be single threaded + * @param queueLatencyRecorder consumer to record queue latency, Long.MAX_VALUE is used for requests that timed out + */ public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis, ScheduledExecutorService executor, LongConsumer queueLatencyRecorder) { this(maxPermits, maxQueueSize, timeoutMillis, executor, false, queueLatencyRecorder); @@ -182,7 +196,7 @@ private void processQueue() { executor.execute(processQueueRunnable); } - private void internalProcessQueue() { + private synchronized void internalProcessQueue() { while (!closed.get()) { long current = availablePermits.get(); if (current <= 0) { From 05b789ee56599233636a0cd830f2dee107a2efd8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 15 Oct 2025 20:31:33 +0300 Subject: [PATCH 30/75] Add check for maxPermits when acquiring permits --- .../common/semaphore/AsyncSemaphoreImpl.java | 4 +-- .../semaphore/AsyncSemaphoreImplTest.java | 29 +++++++++++++++++++ 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index 5e20a07f29b8e..4b594a04ef0b6 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -97,7 +97,7 @@ public CompletableFuture acquire(long permits, BooleanSupp private CompletableFuture internalAcquire(long permits, long acquirePermits, BooleanSupplier isCancelled) { - if (permits < 0) { + if (permits < 0 || permits > maxPermits) { throw new IllegalArgumentException("Invalid permits value: " + permits); } @@ -142,7 +142,7 @@ private void recordQueueLatency(long ageNanos) { @Override public CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, BooleanSupplier isCancelled) { - if (newPermits < 0) { + if (newPermits < 0 || newPermits > maxPermits) { throw new IllegalArgumentException("Invalid permits value: " + newPermits); } long oldPermits = permit.getPermits(); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java index 0331c6aba973d..7fa08097fe15d 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java @@ -190,6 +190,18 @@ public void testInvalidPermits() { } } + @Test + public void testInvalidPermitsExceedingMaxPermits() { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + try { + semaphore.acquire(11, () -> false); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid permits value")); + } + } + @Test public void testClose() throws Exception { semaphore = new AsyncSemaphoreImpl(5, 10, 5000); @@ -281,6 +293,23 @@ public void testUpdateWithInvalidPermits() throws Exception { semaphore.release(permit); } + @Test + public void testUpdateWithInvalidPermitsExceedingMaxPermits() throws Exception { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + CompletableFuture future = semaphore.acquire(5, () -> false); + AsyncSemaphorePermit permit = future.get(1, TimeUnit.SECONDS); + + try { + semaphore.update(permit, 11, () -> false); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid permits value")); + } + + semaphore.release(permit); + } + @Test public void testUpdateWithZeroPermitsShouldReleasePermits() throws Exception { semaphore = new AsyncSemaphoreImpl(10, 10, 5000); From 7f5b461bc1160243ef457679ea268923285adb2d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 15 Oct 2025 20:34:10 +0300 Subject: [PATCH 31/75] Add javadoc for AsyncDualMemoryLimiterImpl constructors --- .../semaphore/AsyncDualMemoryLimiterImpl.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java index 1da47033ebc34..d6b31aa72eeb4 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterImpl.java @@ -37,6 +37,16 @@ public class AsyncDualMemoryLimiterImpl implements AsyncDualMemoryLimiter, AutoC private final AsyncSemaphoreImpl heapLimiter; private final AsyncSemaphoreImpl directLimiter; + /** + * Creates an AsyncDualMemoryLimiterImpl with the given parameters. + * @param maxHeapMemory max heap memory available for allocation + * @param maxHeapQueueSize max number of requests that can be queued for heap memory allocation + * @param heapTimeoutMillis timeout in milliseconds for heap memory allocation + * @param maxDirectMemory max direct memory available for allocation + * @param maxDirectQueueSize max number of requests that can be queued for direct memory allocation + * @param directTimeoutMillis timeout in milliseconds for direct memory allocation + * @param executor executor service to use for scheduling timeouts, it is expected to be single threaded + */ public AsyncDualMemoryLimiterImpl(long maxHeapMemory, int maxHeapQueueSize, long heapTimeoutMillis, long maxDirectMemory, int maxDirectQueueSize, long directTimeoutMillis, ScheduledExecutorService executor) { @@ -44,6 +54,15 @@ public AsyncDualMemoryLimiterImpl(long maxHeapMemory, int maxHeapQueueSize, long directTimeoutMillis, executor, false); } + /** + * Creates an AsyncDualMemoryLimiterImpl with the given parameters. + * @param maxHeapMemory max heap memory available for allocation + * @param maxHeapQueueSize max number of requests that can be queued for heap memory allocation + * @param heapTimeoutMillis timeout in milliseconds for heap memory allocation + * @param maxDirectMemory max direct memory available for allocation + * @param maxDirectQueueSize max number of requests that can be queued for direct memory allocation + * @param directTimeoutMillis timeout in milliseconds for direct memory allocation + */ public AsyncDualMemoryLimiterImpl(long maxHeapMemory, int maxHeapQueueSize, long heapTimeoutMillis, long maxDirectMemory, int maxDirectQueueSize, long directTimeoutMillis) { this(maxHeapMemory, maxHeapQueueSize, heapTimeoutMillis, maxDirectMemory, maxDirectQueueSize, From ea5a7e06862a80fd31550cf64e302ac973db0de3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 15 Oct 2025 20:34:10 +0300 Subject: [PATCH 32/75] Add topic list size cache to address heap limit with proxy --- .../TopicListSizeResultCache.java | 57 +++++++++++++++++++ .../pulsar/broker/service/BrokerService.java | 3 + .../pulsar/broker/service/ServerCnx.java | 10 +++- .../proxy/server/LookupProxyHandler.java | 17 ++++-- .../pulsar/proxy/server/ProxyService.java | 4 ++ 5 files changed, 83 insertions(+), 8 deletions(-) create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java new file mode 100644 index 0000000000000..a1c75b831b6b9 --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.topiclistlimit; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; + +public class TopicListSizeResultCache { + // 1KB initial estimate for topic list heap size + private static final long INITIAL_TOPIC_LIST_HEAP_SIZE = 1024; + + private Cache topicListSizeCache = Caffeine.newBuilder() + .maximumSize(10000) + .build(); + + record CacheKey(String namespaceName, CommandGetTopicsOfNamespace.Mode mode) { + } + + public long getTopicListSize(String namespaceName, CommandGetTopicsOfNamespace.Mode mode) { + Long topicListSize = topicListSizeCache.getIfPresent(new CacheKey(namespaceName, mode)); + return topicListSize != null ? topicListSize : INITIAL_TOPIC_LIST_HEAP_SIZE; + } + + public void updateTopicListSize(String namespaceName, CommandGetTopicsOfNamespace.Mode mode, long recentSize) { + topicListSizeCache.asMap().compute(new CacheKey(namespaceName, mode), (k, existingSize) -> { + if (existingSize != null) { + // update by calculate the average recentSize of existing and the new recentSize + long updatedSize = (existingSize + recentSize) / 2; + // if the difference is more than 1, update the size + if (Math.abs(updatedSize - existingSize) > 1) { + return updatedSize; + } else { + return existingSize; + } + } else { + return recentSize; + } + }); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index e173f8d64a239..05918ce664276 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -140,6 +140,7 @@ import org.apache.pulsar.broker.storage.ManagedLedgerStorage; import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; +import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; import org.apache.pulsar.broker.validator.BindAddressValidator; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -254,6 +255,8 @@ public class BrokerService implements Closeable { protected final AtomicReference lookupRequestSemaphore; @Getter private final AsyncDualMemoryLimiterImpl maxTopicListInFlightLimiter; + @Getter + private final TopicListSizeResultCache topicListSizeResultCache = new TopicListSizeResultCache(); protected final AtomicReference topicLoadRequestSemaphore; public static final String TOPIC_LOOKUP_USAGE_METRIC_NAME = "pulsar.broker.request.topic.lookup.concurrent.usage"; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index eeee6a36c71bb..c1bf342850a2b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -198,8 +198,6 @@ * parameter instance lifecycle. */ public class ServerCnx extends PulsarHandler implements TransportCnx { - // 1KB initial estimate for topic list heap permits size - private static final long INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE = 1024; private static final Logger PAUSE_RECEIVING_LOG = LoggerFactory.getLogger(ServerCnx.class.getName() + ".pauseReceiving"); private final BrokerService service; @@ -2583,7 +2581,8 @@ private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName Optional topicsPattern, Optional topicsHash, Semaphore lookupSemaphore) { BooleanSupplier isPermitRequestCancelled = () -> !ctx().channel().isActive(); - maxTopicListInFlightLimiter.withAcquiredPermits(INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, + long initialSize = service.getTopicListSizeResultCache().getTopicListSize(namespaceName.toString(), mode); + maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { return getBrokerService().pulsar().getNamespaceService().getListOfUserTopics(namespaceName, mode) .thenAccept(topics -> { @@ -2591,6 +2590,11 @@ private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes .map(n -> n + 32) // add 32 bytes overhead for each entry .sum(); + // update the cached size if there's a difference larger than 1 + if (Math.abs(initialSize - actualSize) > 1) { + service.getTopicListSizeResultCache() + .updateTopicListSize(namespaceName.toString(), mode, actualSize); + } maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, isPermitRequestCancelled, permits -> { boolean filterTopics = false; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java index 2fe3fd2d923a1..529ff355cd1c3 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java @@ -32,6 +32,7 @@ import java.util.concurrent.Semaphore; import java.util.function.BooleanSupplier; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.BinaryProtoLookupService; import org.apache.pulsar.client.impl.ClientCnx; @@ -55,8 +56,6 @@ import org.slf4j.LoggerFactory; public class LookupProxyHandler { - // 1KB initial estimate for topic list heap permits size - private static final long INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE = 1024; private final String throttlingErrorMessage = "Too many concurrent lookup and partitionsMetadata requests"; private final ProxyConnection proxyConnection; private final BrokerDiscoveryProvider discoveryProvider; @@ -96,6 +95,7 @@ public class LookupProxyHandler { .create().register(); private final Semaphore lookupRequestSemaphore; private final AsyncDualMemoryLimiterImpl maxTopicListInFlightLimiter; + private final TopicListSizeResultCache topicListSizeResultCache; public LookupProxyHandler(ProxyService proxy, ProxyConnection proxyConnection) { this.discoveryProvider = proxy.getDiscoveryProvider(); @@ -106,6 +106,7 @@ public LookupProxyHandler(ProxyService proxy, ProxyConnection proxyConnection) { this.brokerServiceURL = this.connectWithTLS ? proxy.getConfiguration().getBrokerServiceURLTLS() : proxy.getConfiguration().getBrokerServiceURL(); this.maxTopicListInFlightLimiter = proxy.getMaxTopicListInFlightLimiter(); + this.topicListSizeResultCache = proxy.getTopicListSizeResultCache(); } public void handleLookup(CommandLookupTopic lookup) { @@ -348,7 +349,7 @@ private void performGetTopicsOfNamespace(long clientRequestId, command = Commands.newGetTopicsOfNamespaceRequest(namespaceName, requestId, mode, topicsPattern, topicsHash); - internalPerformGetTopicsOfNamespace(clientRequestId, namespaceName, clientCnx, command, requestId); + internalPerformGetTopicsOfNamespace(clientRequestId, namespaceName, mode, clientCnx, command, requestId); proxyConnection.getConnectionPool().releaseConnection(clientCnx); }).exceptionally(ex -> { // Failed to connect to backend broker @@ -358,10 +359,12 @@ private void performGetTopicsOfNamespace(long clientRequestId, }); } - private void internalPerformGetTopicsOfNamespace(long clientRequestId, String namespaceName, ClientCnx clientCnx, + private void internalPerformGetTopicsOfNamespace(long clientRequestId, String namespaceName, + CommandGetTopicsOfNamespace.Mode mode, ClientCnx clientCnx, ByteBuf command, long requestId) { BooleanSupplier isPermitRequestCancelled = () -> !proxyConnection.ctx().channel().isActive(); - maxTopicListInFlightLimiter.withAcquiredPermits(INITIAL_TOPIC_LIST_HEAP_PERMITS_SIZE, + long initialSize = topicListSizeResultCache.getTopicListSize(namespaceName, mode); + maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { return clientCnx.newGetTopicsOfNamespace(command, requestId).whenComplete((r, t) -> { if (t != null) { @@ -374,6 +377,10 @@ private void internalPerformGetTopicsOfNamespace(long clientRequestId, String na .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes .map(n -> n + 32) // add 32 bytes overhead for each entry .sum(); + // update the cached size if there's a difference larger than 1 + if (Math.abs(initialSize - actualSize) > 1) { + topicListSizeResultCache.updateTopicListSize(namespaceName, mode, actualSize); + } maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, isPermitRequestCancelled, permits -> { return handleWritingGetTopicsResponse(clientRequestId, r, isPermitRequestCancelled); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index d8aac78a97e01..251e25a0e6bc3 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -64,6 +64,7 @@ import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; +import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlets; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -160,6 +161,9 @@ public class ProxyService implements Closeable { @Getter private final AsyncDualMemoryLimiterImpl maxTopicListInFlightLimiter; + @Getter + private final TopicListSizeResultCache topicListSizeResultCache = new TopicListSizeResultCache(); + private boolean gracefulShutdown = true; public ProxyService(ProxyConfiguration proxyConfig, From a2bb80612570fd18df4f6acdeed1d5f1c7ba48ae Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 15 Oct 2025 22:38:14 +0300 Subject: [PATCH 33/75] Don't release the heap permit before direct memory permit has been acquired --- .../pulsar/broker/service/PulsarCommandSender.java | 4 ++-- .../broker/service/PulsarCommandSenderImpl.java | 8 ++++---- .../org/apache/pulsar/broker/service/ServerCnx.java | 11 ++++++----- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java index e0ef1b9011681..5931507c8d8c6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java @@ -53,8 +53,8 @@ void sendSendReceiptResponse(long producerId, long sequenceId, long highestId, l void sendSendError(long producerId, long sequenceId, ServerError error, String errorMsg); - void sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, boolean filtered, boolean changed, - long requestId, Consumer permitAcquireErrorHandler); + CompletableFuture sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, boolean filtered, boolean changed, + long requestId, Consumer permitAcquireErrorHandler); void sendGetSchemaResponse(long requestId, SchemaInfo schema, SchemaVersion version); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java index 6d619123a213a..32b699f69cdb7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java @@ -127,13 +127,13 @@ public void sendSendError(long producerId, long sequenceId, ServerError error, S } @Override - public void sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, - boolean filtered, boolean changed, long requestId, - Consumer permitAcquireErrorHandler) { + public CompletableFuture sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, + boolean filtered, boolean changed, long requestId, + Consumer permitAcquireErrorHandler) { BaseCommand command = Commands.newGetTopicsOfNamespaceResponseCommand(topics, topicsHash, filtered, changed, requestId); safeIntercept(command, cnx); - acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), + return acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), command, permitAcquireErrorHandler); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index c1bf342850a2b..f4cc63d978ef2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -2623,14 +2623,15 @@ private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName requestId, topics.size()); } - commandSender.sendGetTopicsOfNamespaceResponse(filteredTopics, hash, filterTopics, - !hashUnchanged, requestId, ex -> { + return commandSender.sendGetTopicsOfNamespaceResponse(filteredTopics, hash, + filterTopics, !hashUnchanged, requestId, ex -> { log.warn("[{}] Failed to acquire direct memory permits for " - + "GetTopicsOfNamespace: {}", remoteAddress, ex.getMessage()); - commandSender.sendErrorResponse(requestId, ServerError.TooManyRequests, + + "GetTopicsOfNamespace: {}", remoteAddress, + ex.getMessage()); + commandSender.sendErrorResponse(requestId, + ServerError.TooManyRequests, "Cannot acquire permits for direct memory"); }); - return CompletableFuture.completedFuture(null); }, t -> { log.warn("[{}] Failed to acquire heap memory permits for " + "GetTopicsOfNamespace: {}", remoteAddress, t.getMessage()); From db17938e58bc726fcdc02e82e0e60f888c04a252 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 15 Oct 2025 22:38:50 +0300 Subject: [PATCH 34/75] Increase concurrency in test --- .../api/PatternConsumerBackPressureMultipleConsumersTest.java | 2 +- .../ProxyPatternConsumerBackPressureMultipleConsumersTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java index 2acbf38fe37e7..da122718c430e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java @@ -77,7 +77,7 @@ public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() // number of topics to create final int topicCount = 8192; // maximum number of requests in flight at any given time - final int maxRequestsInFlight = 200; + final int maxRequestsInFlight = 500; // create a single topic with multiple partitions admin.topics().createPartitionedTopic(topicName, topicCount); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java index abf43a568de0a..3263cc2985c4b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPatternConsumerBackPressureMultipleConsumersTest.java @@ -82,7 +82,7 @@ protected String getClientServiceUrl() { @Override protected int getDirectMemoryRequiredMB() { - return 225; + return 2 * super.getDirectMemoryRequiredMB(); } @Override From 591b8d2ddd77535f98e428777bf9bf3456e6edcd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 15 Oct 2025 23:31:20 +0300 Subject: [PATCH 35/75] Add solution that performs a single request for getting the size of the topic listing before proceeding --- .../TopicListSizeResultCache.java | 78 ++++++--- .../pulsar/broker/service/ServerCnx.java | 158 ++++++++++-------- .../proxy/server/LookupProxyHandler.java | 88 +++++----- 3 files changed, 192 insertions(+), 132 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java index a1c75b831b6b9..085f08cb960d0 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java @@ -20,38 +20,78 @@ import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; public class TopicListSizeResultCache { - // 1KB initial estimate for topic list heap size - private static final long INITIAL_TOPIC_LIST_HEAP_SIZE = 1024; + // 10kB initial estimate for topic list heap size + private static final long INITIAL_TOPIC_LIST_HEAP_SIZE = 10 * 1024; - private Cache topicListSizeCache = Caffeine.newBuilder() + private Cache topicListSizeCache = Caffeine.newBuilder() .maximumSize(10000) .build(); record CacheKey(String namespaceName, CommandGetTopicsOfNamespace.Mode mode) { } - public long getTopicListSize(String namespaceName, CommandGetTopicsOfNamespace.Mode mode) { - Long topicListSize = topicListSizeCache.getIfPresent(new CacheKey(namespaceName, mode)); - return topicListSize != null ? topicListSize : INITIAL_TOPIC_LIST_HEAP_SIZE; - } + public static class ResultHolder { + private final AtomicReference> topicListSizeFuture = + new AtomicReference<>(null); + private final AtomicLong existingSizeRef = new AtomicLong(-1L); + + public CompletableFuture getSizeAsync() { + if (topicListSizeFuture.compareAndSet(null, new CompletableFuture<>())) { + // let the first request proceed with the initial estimate + return CompletableFuture.completedFuture(INITIAL_TOPIC_LIST_HEAP_SIZE); + } else { + // all other requests wait for the first one to complete + return topicListSizeFuture.get(); + } + } - public void updateTopicListSize(String namespaceName, CommandGetTopicsOfNamespace.Mode mode, long recentSize) { - topicListSizeCache.asMap().compute(new CacheKey(namespaceName, mode), (k, existingSize) -> { - if (existingSize != null) { - // update by calculate the average recentSize of existing and the new recentSize - long updatedSize = (existingSize + recentSize) / 2; - // if the difference is more than 1, update the size - if (Math.abs(updatedSize - existingSize) > 1) { - return updatedSize; + public void updateSize(long actualSize) { + long existingSizeValue = existingSizeRef.updateAndGet(existingSize -> { + if (existingSize > 0) { + // update by calculate the average actualSize of existing and the new actualSize + long updatedSize = (existingSize + actualSize) / 2; + // if the difference is more than 1, update the size + if (Math.abs(updatedSize - existingSize) > 1) { + return updatedSize; + } else { + return existingSize; + } } else { - return existingSize; + return actualSize; } - } else { - return recentSize; + }); + CompletableFuture currentFuture = topicListSizeFuture.get(); + if (currentFuture != null && !currentFuture.isDone()) { + currentFuture.complete(existingSizeValue); + } else if (currentFuture == null || currentFuture.join().longValue() != existingSizeValue) { + topicListSizeFuture.compareAndSet(currentFuture, CompletableFuture.completedFuture(existingSizeValue)); } - }); + } + + public void resetIfInitializing() { + CompletableFuture currentFuture = topicListSizeFuture.getAndUpdate(value -> { + if (value != null && !value.isDone()) { + return null; + } else { + return value; + } + }); + // let all current requests proceed with the initial estimate if this were to happen + if (currentFuture != null && !currentFuture.isDone()) { + currentFuture.complete(INITIAL_TOPIC_LIST_HEAP_SIZE); + } + } + } + + public ResultHolder getTopicListSize(String namespaceName, + CommandGetTopicsOfNamespace.Mode mode) { + return topicListSizeCache.asMap() + .computeIfAbsent(new CacheKey(namespaceName, mode), __ -> new ResultHolder()); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index f4cc63d978ef2..086fc3d7a3a52 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -100,6 +100,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; @@ -2581,82 +2582,93 @@ private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName Optional topicsPattern, Optional topicsHash, Semaphore lookupSemaphore) { BooleanSupplier isPermitRequestCancelled = () -> !ctx().channel().isActive(); - long initialSize = service.getTopicListSizeResultCache().getTopicListSize(namespaceName.toString(), mode); - maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, - AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { - return getBrokerService().pulsar().getNamespaceService().getListOfUserTopics(namespaceName, mode) - .thenAccept(topics -> { - long actualSize = topics.stream() - .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes - .map(n -> n + 32) // add 32 bytes overhead for each entry - .sum(); - // update the cached size if there's a difference larger than 1 - if (Math.abs(initialSize - actualSize) > 1) { - service.getTopicListSizeResultCache() - .updateTopicListSize(namespaceName.toString(), mode, actualSize); - } - maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, - isPermitRequestCancelled, permits -> { - boolean filterTopics = false; - // filter system topic - List filteredTopics = topics; - - if (enableSubscriptionPatternEvaluation && topicsPattern.isPresent()) { - if (topicsPattern.get().length() <= maxSubscriptionPatternLength) { - filterTopics = true; - filteredTopics = TopicList.filterTopics(filteredTopics, topicsPattern.get(), - topicsPatternImplementation); - } else { - log.info("[{}] Subscription pattern provided [{}] was longer than " - + "maximum {}.", remoteAddress, topicsPattern.get(), - maxSubscriptionPatternLength); - } + TopicListSizeResultCache.ResultHolder + listSizeHolder = service.getTopicListSizeResultCache().getTopicListSize(namespaceName.toString(), mode); + listSizeHolder.getSizeAsync().thenAccept(initialSize -> { + maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { + return getBrokerService().pulsar().getNamespaceService() + .getListOfUserTopics(namespaceName, mode) + .thenAccept(topics -> { + long actualSize = topics.stream() + .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes + .map(n -> n + 32) // add 32 bytes overhead for each entry + .sum(); + // update the cached size if there's a difference larger than 1 + if (Math.abs(initialSize - actualSize) > 1) { + listSizeHolder.updateSize(actualSize); } - String hash = TopicList.calculateHash(filteredTopics); - boolean hashUnchanged = topicsHash.isPresent() && topicsHash.get().equals(hash); - if (hashUnchanged) { - filteredTopics = Collections.emptyList(); - } - if (log.isDebugEnabled()) { - log.debug("[{}] Received CommandGetTopicsOfNamespace for namespace " - + "[//{}] by {}, size:{}", remoteAddress, namespace, - requestId, - topics.size()); - } - return commandSender.sendGetTopicsOfNamespaceResponse(filteredTopics, hash, - filterTopics, !hashUnchanged, requestId, ex -> { - log.warn("[{}] Failed to acquire direct memory permits for " - + "GetTopicsOfNamespace: {}", remoteAddress, - ex.getMessage()); - commandSender.sendErrorResponse(requestId, - ServerError.TooManyRequests, - "Cannot acquire permits for direct memory"); + maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, + isPermitRequestCancelled, permits -> { + boolean filterTopics = false; + // filter system topic + List filteredTopics = topics; + + if (enableSubscriptionPatternEvaluation && topicsPattern.isPresent()) { + if (topicsPattern.get().length() <= maxSubscriptionPatternLength) { + filterTopics = true; + filteredTopics = TopicList.filterTopics(filteredTopics, + topicsPattern.get(), + topicsPatternImplementation); + } else { + log.info( + "[{}] Subscription pattern provided [{}] was longer " + + "than maximum {}.", remoteAddress, + topicsPattern.get(), + maxSubscriptionPatternLength); + } + } + String hash = TopicList.calculateHash(filteredTopics); + boolean hashUnchanged = + topicsHash.isPresent() && topicsHash.get().equals(hash); + if (hashUnchanged) { + filteredTopics = Collections.emptyList(); + } + if (log.isDebugEnabled()) { + log.debug("[{}] Received CommandGetTopicsOfNamespace for namespace " + + "[//{}] by {}, size:{}", remoteAddress, namespace, + requestId, + topics.size()); + } + return commandSender.sendGetTopicsOfNamespaceResponse(filteredTopics, + hash, + filterTopics, !hashUnchanged, requestId, ex -> { + log.warn("[{}] Failed to acquire direct memory permits for " + + "GetTopicsOfNamespace: {}", remoteAddress, + ex.getMessage()); + commandSender.sendErrorResponse(requestId, + ServerError.TooManyRequests, + "Cannot acquire permits for direct memory"); + }); + }, t -> { + log.warn("[{}] Failed to acquire heap memory permits for " + + "GetTopicsOfNamespace: {}", remoteAddress, t.getMessage()); + writeAndFlush(Commands.newError(requestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + return CompletableFuture.completedFuture(null); }); - }, t -> { - log.warn("[{}] Failed to acquire heap memory permits for " - + "GetTopicsOfNamespace: {}", remoteAddress, t.getMessage()); - writeAndFlush(Commands.newError(requestId, ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); - return CompletableFuture.completedFuture(null); + }).whenComplete((__, ___) -> { + lookupSemaphore.release(); + }).exceptionally(ex -> { + log.warn("[{}] Error GetTopicsOfNamespace for namespace [//{}] by {}", + remoteAddress, + namespace, requestId); + listSizeHolder.resetIfInitializing(); + commandSender.sendErrorResponse(requestId, + BrokerServiceException.getClientErrorCode(new ServerMetadataException(ex)), + ex.getMessage()); + return null; }); - }).whenComplete((__, ___) -> { - lookupSemaphore.release(); - }).exceptionally(ex -> { - log.warn("[{}] Error GetTopicsOfNamespace for namespace [//{}] by {}", remoteAddress, - namespace, requestId); - commandSender.sendErrorResponse(requestId, - BrokerServiceException.getClientErrorCode(new ServerMetadataException(ex)), - ex.getMessage()); - return null; - }); - }, t -> { - log.warn("[{}] Failed to acquire initial heap memory permits for GetTopicsOfNamespace: {}", - remoteAddress, t.getMessage()); - writeAndFlush(Commands.newError(requestId, ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); - lookupSemaphore.release(); - return CompletableFuture.completedFuture(null); - }); + }, t -> { + log.warn("[{}] Failed to acquire initial heap memory permits for GetTopicsOfNamespace: {}", + remoteAddress, t.getMessage()); + listSizeHolder.resetIfInitializing(); + writeAndFlush(Commands.newError(requestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + lookupSemaphore.release(); + return CompletableFuture.completedFuture(null); + }); + }); } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java index 529ff355cd1c3..df109b25d8409 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java @@ -363,47 +363,55 @@ private void internalPerformGetTopicsOfNamespace(long clientRequestId, String na CommandGetTopicsOfNamespace.Mode mode, ClientCnx clientCnx, ByteBuf command, long requestId) { BooleanSupplier isPermitRequestCancelled = () -> !proxyConnection.ctx().channel().isActive(); - long initialSize = topicListSizeResultCache.getTopicListSize(namespaceName, mode); - maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, - AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { - return clientCnx.newGetTopicsOfNamespace(command, requestId).whenComplete((r, t) -> { - if (t != null) { - log.warn("[{}] Failed to get TopicsOfNamespace {}: {}", clientAddress, namespaceName, - t.getMessage()); - writeAndFlush(Commands.newError(clientRequestId, getServerError(t), t.getMessage())); - } else { - long actualSize = - r.getNonPartitionedOrPartitionTopics().stream() - .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes - .map(n -> n + 32) // add 32 bytes overhead for each entry - .sum(); - // update the cached size if there's a difference larger than 1 - if (Math.abs(initialSize - actualSize) > 1) { - topicListSizeResultCache.updateTopicListSize(namespaceName, mode, actualSize); + TopicListSizeResultCache.ResultHolder + listSizeHolder = topicListSizeResultCache.getTopicListSize(namespaceName, mode); + listSizeHolder.getSizeAsync().thenAccept(initialSize -> { + maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { + return clientCnx.newGetTopicsOfNamespace(command, requestId).whenComplete((r, t) -> { + if (t != null) { + log.warn("[{}] Failed to get TopicsOfNamespace {}: {}", clientAddress, namespaceName, + t.getMessage()); + listSizeHolder.resetIfInitializing(); + writeAndFlush(Commands.newError(clientRequestId, getServerError(t), t.getMessage())); + } else { + long actualSize = + r.getNonPartitionedOrPartitionTopics().stream() + .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes + .map(n -> n + 32) // add 32 bytes overhead for each entry + .sum(); + // update the cached size if there's a difference larger than 1 + if (Math.abs(initialSize - actualSize) > 1) { + listSizeHolder.updateSize(actualSize); + } + maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, + isPermitRequestCancelled, permits -> { + return handleWritingGetTopicsResponse(clientRequestId, r, + isPermitRequestCancelled); + }, t2 -> { + log.warn("[{}] Failed to acquire actual heap memory permits for " + + "GetTopicsOfNamespace: {}", clientAddress, t2.getMessage()); + writeAndFlush( + Commands.newError(clientRequestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + + return CompletableFuture.completedFuture(null); + }); } - maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, - isPermitRequestCancelled, permits -> { - return handleWritingGetTopicsResponse(clientRequestId, r, isPermitRequestCancelled); - }, t2 -> { - log.warn("[{}] Failed to acquire actual heap memory permits for " - + "GetTopicsOfNamespace: {}", clientAddress, t2.getMessage()); - writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); - - return CompletableFuture.completedFuture(null); - }); - } - }).thenApply(__ -> null); - }, t -> { - log.warn("[{}] Failed to acquire initial heap memory permits for GetTopicsOfNamespace: {}", - clientAddress, t.getMessage()); - writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, - "Failed due to heap memory limit exceeded")); - - return CompletableFuture.completedFuture(null); - }).exceptionally(ex -> { - writeAndFlush(Commands.newError(clientRequestId, getServerError(ex), ex.getMessage())); - return null; + }).thenApply(__ -> null); + }, t -> { + log.warn("[{}] Failed to acquire initial heap memory permits for GetTopicsOfNamespace: {}", + clientAddress, t.getMessage()); + listSizeHolder.resetIfInitializing(); + writeAndFlush(Commands.newError(clientRequestId, ServerError.TooManyRequests, + "Failed due to heap memory limit exceeded")); + + return CompletableFuture.completedFuture(null); + }).exceptionally(ex -> { + listSizeHolder.resetIfInitializing(); + writeAndFlush(Commands.newError(clientRequestId, getServerError(ex), ex.getMessage())); + return null; + }); }); } From 7b93184cdce8832abe74fccd9793932eb4202435 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 15 Oct 2025 23:43:38 +0300 Subject: [PATCH 36/75] Checkstyle --- .../apache/pulsar/broker/service/PulsarCommandSender.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java index 5931507c8d8c6..7b144340c3191 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java @@ -53,8 +53,9 @@ void sendSendReceiptResponse(long producerId, long sequenceId, long highestId, l void sendSendError(long producerId, long sequenceId, ServerError error, String errorMsg); - CompletableFuture sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, boolean filtered, boolean changed, - long requestId, Consumer permitAcquireErrorHandler); + CompletableFuture sendGetTopicsOfNamespaceResponse(List topics, String topicsHash, boolean filtered, + boolean changed, long requestId, + Consumer permitAcquireErrorHandler); void sendGetSchemaResponse(long requestId, SchemaInfo schema, SchemaVersion version); From 0640095d4724c4ce3eb0637c849b877250c3f40f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 16 Oct 2025 00:08:51 +0300 Subject: [PATCH 37/75] Add javadoc for TopicListSizeResultCache --- .../TopicListSizeResultCache.java | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java index 085f08cb960d0..e6951b00be93f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java @@ -25,6 +25,17 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; +/** + * A cache for storing and managing topic list size estimates in namespaces. + * This class provides functionality to: + * - Cache and retrieve topic list size estimates for different namespaces and modes + * - Handle concurrent requests for topic list sizes efficiently + * - Maintain and update size estimates based on actual topic list sizes + * - Prevent thundering herd problems when multiple concurrent requests for a namespace are made without + * a previous size estimate + * The cache uses namespace name and topic list mode (PERSISTENT/NON_PERSISTENT/ALL) as keys + * and maintains size estimates that are refined with actual usage. + */ public class TopicListSizeResultCache { // 10kB initial estimate for topic list heap size private static final long INITIAL_TOPIC_LIST_HEAP_SIZE = 10 * 1024; @@ -36,11 +47,24 @@ public class TopicListSizeResultCache { record CacheKey(String namespaceName, CommandGetTopicsOfNamespace.Mode mode) { } + /** + * Holds the topic list size estimate and future for the topic list size. + * The size is returned by calling {@link #getSizeAsync()} method which is asynchronous. + * The estimate is updated by calling {@link #updateSize(long)} method. + */ public static class ResultHolder { private final AtomicReference> topicListSizeFuture = new AtomicReference<>(null); private final AtomicLong existingSizeRef = new AtomicLong(-1L); + /** + * Get the topic list size estimate. The first request will return the initial estimate + * and update the estimate based on the returned size of the topic list. Other concurrent requests + * will wait for the first request to complete and use the estimate of the first request. + * Subsequent requests will use estimate which gets updated based on the returned size of the topic list + * of each request. + * @return a future that will return the topic list size estimate + */ public CompletableFuture getSizeAsync() { if (topicListSizeFuture.compareAndSet(null, new CompletableFuture<>())) { // let the first request proceed with the initial estimate @@ -51,6 +75,12 @@ public CompletableFuture getSizeAsync() { } } + /** + * Update the topic list size estimate. The new estimated size will be updated by calculating the average + * of the existing and the new size. If the difference between the new and the existing size is less than 1, + * no update will be done. + * @param actualSize the actual size of the topic list + */ public void updateSize(long actualSize) { long existingSizeValue = existingSizeRef.updateAndGet(existingSize -> { if (existingSize > 0) { @@ -74,6 +104,10 @@ public void updateSize(long actualSize) { } } + /** + * After errors, it's necessary to call this method to ensure that the instance isn't left in a state + * where concurrent requests are waiting for the first request to complete the future by calling updateSize. + */ public void resetIfInitializing() { CompletableFuture currentFuture = topicListSizeFuture.getAndUpdate(value -> { if (value != null && !value.isDone()) { @@ -89,6 +123,12 @@ public void resetIfInitializing() { } } + /** + * Get the topic list size result holder for the given namespace and mode. + * @param namespaceName the namespace name in the format of "tenant/namespace" + * @param mode the mode of the topic list request (PERSISTENT, NON_PERSISTENT, ALL) + * @return the topic list size result holder + */ public ResultHolder getTopicListSize(String namespaceName, CommandGetTopicsOfNamespace.Mode mode) { return topicListSizeCache.asMap() From 06fff3ab92b6d0fb53f8a29431a476f404967d75 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 16 Oct 2025 01:00:33 +0300 Subject: [PATCH 38/75] Fix test --- .../semaphore/AsyncSemaphoreImplTest.java | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java index 7fa08097fe15d..b64d8c27b3219 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.semaphore; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -356,14 +357,14 @@ public void testConcurrentAcquireAndRelease() throws Exception { @Test public void testTimeoutProcessesNextRequest() throws Exception { - semaphore = new AsyncSemaphoreImpl(5, 10, 500); + semaphore = new AsyncSemaphoreImpl(5, 10, 250); // Acquire all permits CompletableFuture future1 = semaphore.acquire(5, () -> false); AsyncSemaphorePermit permit1 = future1.get(1, TimeUnit.SECONDS); // Request that will timeout (needs more permits than available) - CompletableFuture future2 = semaphore.acquire(10, () -> false); + CompletableFuture future2 = semaphore.acquire(5, () -> false); // make requests 250ms apart Thread.sleep(250); @@ -374,15 +375,17 @@ public void testTimeoutProcessesNextRequest() throws Exception { // Release permits semaphore.release(permit1); - // The smaller request should succeed - AsyncSemaphorePermit permit3 = future3.get(1, TimeUnit.SECONDS); + // Expect the second request to timeout + Throwable throwable = future2.handle((permit, t) -> t).join(); + assertThat(throwable).isInstanceOf(PermitAcquireTimeoutException.class); + + // The third request should have succeeded now + AsyncSemaphorePermit permit3 = future3.join(); assertNotNull(permit3); assertEquals(permit3.getPermits(), 3); - // Verify the first timed out - assertTrue(future2.isCompletedExceptionally()); - semaphore.release(permit3); + } @Test From e009d06e7d84c583c6610a11aab24ed3a0030099 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 16 Oct 2025 01:29:09 +0300 Subject: [PATCH 39/75] Add unit tests for TopicListSizeResultCache --- .../TopicListSizeResultCacheTest.java | 369 ++++++++++++++++++ 1 file changed, 369 insertions(+) create mode 100644 pulsar-broker-common/src/test/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCacheTest.java diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCacheTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCacheTest.java new file mode 100644 index 0000000000000..46beb96f02e8a --- /dev/null +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCacheTest.java @@ -0,0 +1,369 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.topiclistlimit; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNotSame; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertTrue; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class TopicListSizeResultCacheTest { + + private TopicListSizeResultCache cache; + private ExecutorService executorService; + + @BeforeMethod + public void setup() { + cache = new TopicListSizeResultCache(); + executorService = Executors.newFixedThreadPool(10); + } + + @AfterMethod(alwaysRun = true) + public void teardown() { + if (executorService != null) { + executorService.shutdownNow(); + } + } + + @Test + public void testGetTopicListSize_returnsSameInstanceForSameKey() { + String namespace = "tenant/namespace"; + CommandGetTopicsOfNamespace.Mode mode = CommandGetTopicsOfNamespace.Mode.ALL; + + TopicListSizeResultCache.ResultHolder holder1 = cache.getTopicListSize(namespace, mode); + TopicListSizeResultCache.ResultHolder holder2 = cache.getTopicListSize(namespace, mode); + + assertSame(holder1, holder2, "Should return the same ResultHolder instance for the same key"); + } + + @Test + public void testGetTopicListSize_returnsDifferentInstancesForDifferentKeys() { + String namespace1 = "tenant/namespace1"; + String namespace2 = "tenant/namespace2"; + CommandGetTopicsOfNamespace.Mode mode = CommandGetTopicsOfNamespace.Mode.ALL; + + TopicListSizeResultCache.ResultHolder holder1 = cache.getTopicListSize(namespace1, mode); + TopicListSizeResultCache.ResultHolder holder2 = cache.getTopicListSize(namespace2, mode); + + assertNotSame(holder1, holder2, "Should return different ResultHolder instances for different namespaces"); + } + + @Test + public void testGetTopicListSize_returnsDifferentInstancesForDifferentModes() { + String namespace = "tenant/namespace"; + + TopicListSizeResultCache.ResultHolder holder1 = cache.getTopicListSize(namespace, + CommandGetTopicsOfNamespace.Mode.ALL); + TopicListSizeResultCache.ResultHolder holder2 = cache.getTopicListSize(namespace, + CommandGetTopicsOfNamespace.Mode.PERSISTENT); + + assertNotSame(holder1, holder2, "Should return different ResultHolder instances for different modes"); + } + + @Test + public void testResultHolder_firstRequestReturnsInitialEstimate() throws Exception { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + + CompletableFuture sizeFuture = holder.getSizeAsync(); + + assertTrue(sizeFuture.isDone(), "First request should complete immediately"); + assertEquals(sizeFuture.get().longValue(), 10 * 1024L, + "First request should return initial estimate of 10KB"); + } + + @Test + public void testResultHolder_concurrentRequestsWaitForFirstToComplete() throws Exception { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + int numConcurrentRequests = 5; + CountDownLatch startLatch = new CountDownLatch(1); + CountDownLatch completeLatch = new CountDownLatch(numConcurrentRequests); + + List> futures = new ArrayList<>(); + + // Start concurrent requests + for (int i = 0; i < numConcurrentRequests; i++) { + executorService.submit(() -> { + try { + startLatch.await(); + CompletableFuture future = holder.getSizeAsync(); + futures.add(future); + completeLatch.countDown(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + } + + // Trigger all requests + startLatch.countDown(); + completeLatch.await(5, TimeUnit.SECONDS); + + // First request completes immediately with initial estimate + assertTrue(futures.get(0).isDone()); + assertEquals(futures.get(0).get().longValue(), 10 * 1024L); + + // Other requests should be waiting + for (int i = 1; i < numConcurrentRequests; i++) { + assertFalse(futures.get(i).isDone(), "Concurrent request " + i + " should be waiting"); + } + + // Update size to complete waiting requests + long actualSize = 20 * 1024L; + holder.updateSize(actualSize); + + // All waiting requests should now complete with the actual size + for (int i = 1; i < numConcurrentRequests; i++) { + assertTrue(futures.get(i).isDone(), "Request " + i + " should complete after updateSize"); + assertEquals(futures.get(i).get().longValue(), actualSize); + } + } + + @Test + public void testResultHolder_updateSize_firstUpdate() { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + + // Get initial size + holder.getSizeAsync(); + + // Update with actual size + long actualSize = 20 * 1024L; + holder.updateSize(actualSize); + + // Next request should return the actual size + CompletableFuture nextFuture = holder.getSizeAsync(); + assertEquals(nextFuture.join().longValue(), actualSize); + } + + @Test + public void testResultHolder_updateSize_calculatesAverage() { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + + // First update + holder.updateSize(10000L); + CompletableFuture future1 = holder.getSizeAsync(); + assertEquals(future1.join().longValue(), 10000L); + + // Second update - should average + holder.updateSize(20000L); + CompletableFuture future2 = holder.getSizeAsync(); + assertEquals(future2.join().longValue(), 15000L, "Should calculate average: (10000 + 20000) / 2"); + + // Third update - should average again + holder.updateSize(30000L); + CompletableFuture future3 = holder.getSizeAsync(); + assertEquals(future3.join().longValue(), 22500L, "Should calculate average: (15000 + 30000) / 2"); + } + + @Test + public void testResultHolder_updateSize_noUpdateWhenDifferenceIsSmall() { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + + // First update + holder.updateSize(1000L); + assertEquals(holder.getSizeAsync().join().longValue(), 1000L); + + // Second update with small difference (average would be 1001) + holder.updateSize(1002L); + // Should not update because abs(1001 - 1000) <= 1 + assertEquals(holder.getSizeAsync().join().longValue(), 1000L, "Should not update when difference <= 1"); + } + + @Test + public void testResultHolder_updateSize_updatesWhenDifferenceIsLarge() { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + + // First update + holder.updateSize(1000L); + assertEquals(holder.getSizeAsync().join().longValue(), 1000L); + + // Second update with large difference (average would be 1500) + holder.updateSize(2000L); + // Should update because abs(1500 - 1000) > 1 + assertEquals(holder.getSizeAsync().join().longValue(), 1500L, "Should update when difference > 1"); + } + + @Test + public void testResultHolder_updateSize_completesWaitingFuture() throws Exception { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + + // First request gets initial estimate + CompletableFuture first = holder.getSizeAsync(); + assertTrue(first.isDone()); + + // Second request should wait + CompletableFuture second = holder.getSizeAsync(); + assertFalse(second.isDone(), "Second request should be waiting"); + + // Update size should complete the waiting future + long actualSize = 15000L; + holder.updateSize(actualSize); + + assertTrue(second.isDone(), "Update should complete waiting future"); + assertEquals(second.get().longValue(), actualSize); + } + + @Test + public void testResultHolder_resetIfInitializing_resetsUncompletedFuture() throws Exception { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + + // First request gets initial estimate + holder.getSizeAsync(); + + // Second request should wait + CompletableFuture waitingFuture = holder.getSizeAsync(); + assertFalse(waitingFuture.isDone(), "Second request should be waiting"); + + // Reset should complete the waiting future with initial estimate + holder.resetIfInitializing(); + + assertTrue(waitingFuture.isDone(), "Reset should complete waiting future"); + assertEquals(waitingFuture.get().longValue(), 10 * 1024L, + "Reset should complete with initial estimate"); + + // Next request should start fresh + CompletableFuture afterReset = holder.getSizeAsync(); + assertTrue(afterReset.isDone(), "Request after reset should complete immediately"); + assertEquals(afterReset.get().longValue(), 10 * 1024L); + } + + @Test + public void testResultHolder_resetIfInitializing_noEffectOnCompletedFuture() throws Exception { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + + // Get initial size and update + holder.getSizeAsync(); + holder.updateSize(20000L); + + CompletableFuture future = holder.getSizeAsync(); + assertTrue(future.isDone()); + long valueBefore = future.get(); + + // Reset should have no effect on completed future + holder.resetIfInitializing(); + + CompletableFuture futureAfter = holder.getSizeAsync(); + assertEquals(futureAfter.join().longValue(), valueBefore, + "Reset should not affect already completed future"); + } + + @Test + public void testResultHolder_multipleSequentialRequests() throws Exception { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + + // First request + CompletableFuture future1 = holder.getSizeAsync(); + assertEquals(future1.get().longValue(), 10 * 1024L); + + // Update + holder.updateSize(15000L); + + // Second request + CompletableFuture future2 = holder.getSizeAsync(); + assertEquals(future2.get().longValue(), 15000L); + + // Update again + holder.updateSize(25000L); + + // Third request + CompletableFuture future3 = holder.getSizeAsync(); + assertEquals(future3.get().longValue(), 20000L, "Should be average of 15000 and 25000"); + } + + @Test + public void testCacheKey_equalityForSameValues() { + TopicListSizeResultCache.CacheKey key1 = new TopicListSizeResultCache.CacheKey("tenant/ns", + CommandGetTopicsOfNamespace.Mode.ALL); + TopicListSizeResultCache.CacheKey key2 = new TopicListSizeResultCache.CacheKey("tenant/ns", + CommandGetTopicsOfNamespace.Mode.ALL); + + assertEquals(key1, key2, "Keys with same values should be equal"); + assertEquals(key1.hashCode(), key2.hashCode(), "Equal keys should have same hash code"); + } + + @Test + public void testCacheKey_inequalityForDifferentNamespaces() { + TopicListSizeResultCache.CacheKey key1 = new TopicListSizeResultCache.CacheKey("tenant/ns1", + CommandGetTopicsOfNamespace.Mode.ALL); + TopicListSizeResultCache.CacheKey key2 = new TopicListSizeResultCache.CacheKey("tenant/ns2", + CommandGetTopicsOfNamespace.Mode.ALL); + + assertNotEquals(key1, key2, "Keys with different namespaces should not be equal"); + } + + @Test + public void testCacheKey_inequalityForDifferentModes() { + TopicListSizeResultCache.CacheKey key1 = new TopicListSizeResultCache.CacheKey("tenant/ns", + CommandGetTopicsOfNamespace.Mode.ALL); + TopicListSizeResultCache.CacheKey key2 = new TopicListSizeResultCache.CacheKey("tenant/ns", + CommandGetTopicsOfNamespace.Mode.PERSISTENT); + + assertNotEquals(key1, key2, "Keys with different modes should not be equal"); + } + + @Test + public void testHighConcurrency() throws Exception { + TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); + int numThreads = 20; + CountDownLatch startLatch = new CountDownLatch(1); + CountDownLatch completeLatch = new CountDownLatch(numThreads); + + List> futures = new ArrayList<>(); + + // Launch many concurrent requests + for (int i = 0; i < numThreads; i++) { + executorService.submit(() -> { + try { + startLatch.await(); + CompletableFuture future = holder.getSizeAsync(); + synchronized (futures) { + futures.add(future); + } + completeLatch.countDown(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + } + + startLatch.countDown(); + assertTrue(completeLatch.await(5, TimeUnit.SECONDS), "All requests should complete"); + + // Update to complete all waiting requests + holder.updateSize(30000L); + + // Verify all futures complete successfully + for (CompletableFuture future : futures) { + assertNotNull(future.get(5, TimeUnit.SECONDS), "All futures should complete"); + } + } +} \ No newline at end of file From 1b48093f226bf0681276174d44d1c34bbc4d543e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 16 Oct 2025 20:00:06 +0300 Subject: [PATCH 40/75] Support making the semaphore unbounded by setting maxPermits to <=0 --- .../common/semaphore/AsyncSemaphoreImpl.java | 27 ++- .../semaphore/AsyncSemaphoreImplTest.java | 157 ++++++++++++++++++ 2 files changed, 181 insertions(+), 3 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index 4b594a04ef0b6..bcf1b21afc75f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -58,7 +58,7 @@ public class AsyncSemaphoreImpl implements AsyncSemaphore, AutoCloseable { * @param timeoutMillis timeout in milliseconds for acquiring permits */ public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis) { - this(maxPermits, maxQueueSize, timeoutMillis, createExecutor(), true, null); + this(maxPermits, maxQueueSize, timeoutMillis, maxPermits > 0 ? createExecutor() : null, maxPermits > 0, null); } /** @@ -97,10 +97,15 @@ public CompletableFuture acquire(long permits, BooleanSupp private CompletableFuture internalAcquire(long permits, long acquirePermits, BooleanSupplier isCancelled) { - if (permits < 0 || permits > maxPermits) { + if (permits < 0 || (permits > maxPermits && !isUnbounded())) { throw new IllegalArgumentException("Invalid permits value: " + permits); } + // if maximum permits is <= 0, then the semaphore is unbounded + if (isUnbounded()) { + return CompletableFuture.completedFuture(new SemaphorePermit(permits)); + } + CompletableFuture future = new CompletableFuture<>(); if (closed.get()) { @@ -133,6 +138,10 @@ private CompletableFuture internalAcquire(long permits, lo return future; } + private boolean isUnbounded() { + return maxPermits <= 0; + } + private void recordQueueLatency(long ageNanos) { if (queueLatencyRecorder != null) { queueLatencyRecorder.accept(ageNanos); @@ -142,9 +151,12 @@ private void recordQueueLatency(long ageNanos) { @Override public CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, BooleanSupplier isCancelled) { - if (newPermits < 0 || newPermits > maxPermits) { + if (newPermits < 0 || (newPermits > maxPermits && !isUnbounded())) { throw new IllegalArgumentException("Invalid permits value: " + newPermits); } + if (isUnbounded()) { + return CompletableFuture.completedFuture(new SemaphorePermit(newPermits)); + } long oldPermits = permit.getPermits(); long additionalPermits = newPermits - oldPermits; // mark the old permits as released without adding the permits to availablePermits @@ -162,17 +174,26 @@ public CompletableFuture update(AsyncSemaphorePermit permi @Override public void release(AsyncSemaphorePermit permit) { + if (isUnbounded()) { + return; + } availablePermits.addAndGet(castToImplementation(permit).releasePermits()); processQueue(); } @Override public long getAvailablePermits() { + if (isUnbounded()) { + return Long.MAX_VALUE; + } return availablePermits.get(); } @Override public long getAcquiredPermits() { + if (isUnbounded()) { + return 0; + } return maxPermits - availablePermits.get(); } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java index b64d8c27b3219..73b62a98fe30d 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java @@ -434,4 +434,161 @@ public void testMultipleReleasesProcessQueue() throws Exception { Thread.sleep(100); assertTrue(queued3.isDone()); } + + @Test + public void testUnboundedSemaphoreAcquireSinglePermit() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + CompletableFuture future = semaphore.acquire(1, () -> false); + AsyncSemaphorePermit permit = future.get(100, TimeUnit.MILLISECONDS); + + assertNotNull(permit); + assertEquals(permit.getPermits(), 1); + assertTrue(future.isDone()); + } + + @Test + public void testUnboundedSemaphoreAcquireMultiplePermits() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + CompletableFuture future = semaphore.acquire(100, () -> false); + AsyncSemaphorePermit permit = future.get(100, TimeUnit.MILLISECONDS); + + assertNotNull(permit); + assertEquals(permit.getPermits(), 100); + assertTrue(future.isDone()); + } + + @Test + public void testUnboundedSemaphoreNoQueueing() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + List> futures = new ArrayList<>(); + + // Acquire many permits concurrently - all should complete immediately + for (int i = 0; i < 100; i++) { + futures.add(semaphore.acquire(10, () -> false)); + } + + // All futures should complete immediately without queueing + for (CompletableFuture future : futures) { + AsyncSemaphorePermit permit = future.get(100, TimeUnit.MILLISECONDS); + assertNotNull(permit); + assertEquals(permit.getPermits(), 10); + assertTrue(future.isDone()); + } + } + + @Test + public void testUnboundedSemaphoreReleaseIsNoop() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + CompletableFuture future = semaphore.acquire(50, () -> false); + AsyncSemaphorePermit permit = future.get(100, TimeUnit.MILLISECONDS); + + // Release should not throw exception but is a no-op + semaphore.release(permit); + + // Should still be able to acquire more permits + CompletableFuture future2 = semaphore.acquire(100, () -> false); + AsyncSemaphorePermit permit2 = future2.get(100, TimeUnit.MILLISECONDS); + assertNotNull(permit2); + } + + @Test + public void testUnboundedSemaphoreAvailablePermits() { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + assertEquals(semaphore.getAvailablePermits(), Long.MAX_VALUE); + } + + @Test + public void testUnboundedSemaphoreAcquiredPermits() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + // Acquire some permits + CompletableFuture future = semaphore.acquire(100, () -> false); + future.get(100, TimeUnit.MILLISECONDS); + + // Acquired permits should always be 0 for unbounded semaphore + assertEquals(semaphore.getAcquiredPermits(), 0); + } + + @Test + public void testUnboundedSemaphoreQueueSize() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + // Acquire multiple permits + for (int i = 0; i < 10; i++) { + semaphore.acquire(1, () -> false).get(100, TimeUnit.MILLISECONDS); + } + + // Queue size should always be 0 since requests complete immediately + assertEquals(semaphore.getQueueSize(), 0); + } + + @Test + public void testUnboundedSemaphoreUpdate() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + CompletableFuture future = semaphore.acquire(10, () -> false); + AsyncSemaphorePermit permit = future.get(100, TimeUnit.MILLISECONDS); + + // Update should complete immediately + CompletableFuture future2 = semaphore.update(permit, 50, () -> false); + AsyncSemaphorePermit permit2 = future2.get(100, TimeUnit.MILLISECONDS); + + assertNotNull(permit2); + assertEquals(permit2.getPermits(), 50); + assertTrue(future2.isDone()); + } + + @Test + public void testUnboundedSemaphoreUpdateToZero() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + CompletableFuture future = semaphore.acquire(100, () -> false); + AsyncSemaphorePermit permit = future.get(100, TimeUnit.MILLISECONDS); + + // Update to zero should complete immediately + CompletableFuture future2 = semaphore.update(permit, 0, () -> false); + AsyncSemaphorePermit permit2 = future2.get(100, TimeUnit.MILLISECONDS); + + assertNotNull(permit2); + assertEquals(permit2.getPermits(), 0); + assertTrue(future2.isDone()); + } + + @Test + public void testUnboundedSemaphoreNoTimeout() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 100); // Short timeout + + // Even with short timeout, requests should complete immediately + CompletableFuture future = semaphore.acquire(1000, () -> false); + AsyncSemaphorePermit permit = future.get(50, TimeUnit.MILLISECONDS); + + assertNotNull(permit); + assertEquals(permit.getPermits(), 1000); + + Thread.sleep(150); // Wait longer than timeout + + // Permit should still be valid + assertEquals(permit.getPermits(), 1000); + } + + @Test + public void testUnboundedSemaphoreCancellationIgnored() throws Exception { + semaphore = new AsyncSemaphoreImpl(0, 10, 5000); + + AtomicBoolean cancelled = new AtomicBoolean(true); + + // Even though cancelled is true, unbounded semaphore completes immediately + CompletableFuture future = semaphore.acquire(10, cancelled::get); + AsyncSemaphorePermit permit = future.get(100, TimeUnit.MILLISECONDS); + + assertNotNull(permit); + assertEquals(permit.getPermits(), 10); + assertTrue(future.isDone()); + assertFalse(future.isCompletedExceptionally()); + } } \ No newline at end of file From 1b07870dc7a5c7b8793ddeb06206e9999d9ec2cb Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 16 Oct 2025 20:02:49 +0300 Subject: [PATCH 41/75] Support disabling max queue size --- .../org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index bcf1b21afc75f..681e6762b6dee 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -23,6 +23,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -78,7 +79,7 @@ public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis, boolean shutdownExecutor, LongConsumer queueLatencyRecorder) { this.availablePermits = new AtomicLong(maxPermits); this.maxPermits = maxPermits; - this.queue = new ArrayBlockingQueue<>(maxQueueSize); + this.queue = maxQueueSize > 0 ? new ArrayBlockingQueue<>(maxQueueSize) : new LinkedBlockingQueue<>(); this.timeoutMillis = timeoutMillis; this.executor = executor; this.shutdownExecutor = shutdownExecutor; From 6163c93b1710c51ce898bcac6ffb5312187ed89a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 16 Oct 2025 20:06:23 +0300 Subject: [PATCH 42/75] Add javadoc for disabling limits --- .../pulsar/common/semaphore/AsyncSemaphoreImpl.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index 681e6762b6dee..5029ef8aadb4c 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -54,8 +54,9 @@ public class AsyncSemaphoreImpl implements AsyncSemaphore, AutoCloseable { /** * Creates an AsyncSemaphoreImpl with the given parameters. - * @param maxPermits max number of permits available for acquisition - * @param maxQueueSize max number of requests that can be queued + * @param maxPermits max number of permits available for acquisition, set to <= 0 for unbounded semaphore (not + * recommended) + * @param maxQueueSize max number of requests that can be queued, set to <= 0 for unbounded queue (not recommended) * @param timeoutMillis timeout in milliseconds for acquiring permits */ public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis) { @@ -64,8 +65,9 @@ public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis) /** * Creates an AsyncSemaphoreImpl with the given parameters. - * @param maxPermits max number of permits available for acquisition - * @param maxQueueSize max number of requests that can be queued + * @param maxPermits max number of permits available for acquisition, set to <= 0 for unbounded semaphore (not + * recommended) + * @param maxQueueSize max number of requests that can be queued, set to <= 0 for unbounded queue (not recommended) * @param timeoutMillis timeout in milliseconds for acquiring permits * @param executor executor service to use for scheduling timeouts, it is expected to be single threaded * @param queueLatencyRecorder consumer to record queue latency, Long.MAX_VALUE is used for requests that timed out From 27f2e9ae8f958959ff1d6ddb41298c7b0dcf077f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 17 Oct 2025 11:34:16 +0300 Subject: [PATCH 43/75] Refactor --- .../pulsar/common/semaphore/AsyncSemaphoreImpl.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index 5029ef8aadb4c..775e95178eb7c 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -100,7 +100,7 @@ public CompletableFuture acquire(long permits, BooleanSupp private CompletableFuture internalAcquire(long permits, long acquirePermits, BooleanSupplier isCancelled) { - if (permits < 0 || (permits > maxPermits && !isUnbounded())) { + if (!isPermitsValidForAcquiring(permits)) { throw new IllegalArgumentException("Invalid permits value: " + permits); } @@ -141,6 +141,10 @@ private CompletableFuture internalAcquire(long permits, lo return future; } + private boolean isPermitsValidForAcquiring(long permits) { + return permits >= 0 && (isUnbounded() || permits <= maxPermits); + } + private boolean isUnbounded() { return maxPermits <= 0; } @@ -154,7 +158,7 @@ private void recordQueueLatency(long ageNanos) { @Override public CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, BooleanSupplier isCancelled) { - if (newPermits < 0 || (newPermits > maxPermits && !isUnbounded())) { + if (!isPermitsValidForAcquiring(newPermits)) { throw new IllegalArgumentException("Invalid permits value: " + newPermits); } if (isUnbounded()) { From 7f11f0358bfe49fe48a5b39e0b00974303f813b3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 17 Oct 2025 16:49:02 +0300 Subject: [PATCH 44/75] Extract estimateTopicListSize utility method --- .../topiclistlimit/TopicListMemoryLimiter.java | 14 ++++++++++++++ .../apache/pulsar/broker/service/ServerCnx.java | 12 +++--------- .../pulsar/proxy/server/LookupProxyHandler.java | 14 ++++---------- 3 files changed, 21 insertions(+), 19 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java index ea0c6bad3b31d..72fad7f4c8ae9 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.topiclistlimit; +import io.netty.buffer.ByteBufUtil; import io.opentelemetry.api.metrics.DoubleGauge; import io.opentelemetry.api.metrics.DoubleHistogram; import io.opentelemetry.api.metrics.LongCounter; @@ -30,6 +31,7 @@ import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import io.prometheus.client.Summary; +import java.util.List; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; @@ -293,4 +295,16 @@ public void close() { otelHeapQueueSize.close(); otelDirectQueueSize.close(); } + + /** + * Estimate the heap memory size of a topic list. + * @param topicList the topic list to estimate + * @return the estimated heap memory size in bytes + */ + public static long estimateTopicListSize(List topicList) { + return topicList.stream() + .mapToLong(ByteBufUtil::utf8Bytes) // convert character count to bytes + .map(n -> n + 32) // add 32 bytes overhead for each entry + .sum(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 6da3988b2a8f1..55cd0fe7e3ddb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -35,7 +35,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufUtil; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelOption; @@ -101,6 +100,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.client.api.PulsarClientException; @@ -2594,14 +2594,8 @@ private void internalHandleGetTopicsOfNamespace(String namespace, NamespaceName return getBrokerService().pulsar().getNamespaceService() .getListOfUserTopics(namespaceName, mode) .thenAccept(topics -> { - long actualSize = topics.stream() - .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes - .map(n -> n + 32) // add 32 bytes overhead for each entry - .sum(); - // update the cached size if there's a difference larger than 1 - if (Math.abs(initialSize - actualSize) > 1) { - listSizeHolder.updateSize(actualSize); - } + long actualSize = TopicListMemoryLimiter.estimateTopicListSize(topics); + listSizeHolder.updateSize(actualSize); maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, isPermitRequestCancelled, permits -> { boolean filterTopics = false; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java index df109b25d8409..49a59444b6271 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java @@ -20,7 +20,6 @@ import static org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterUtil.acquireDirectMemoryPermitsAndWriteAndFlush; import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufUtil; import io.netty.channel.ChannelHandlerContext; import io.prometheus.client.Counter; import java.net.InetSocketAddress; @@ -32,6 +31,7 @@ import java.util.concurrent.Semaphore; import java.util.function.BooleanSupplier; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.BinaryProtoLookupService; @@ -375,15 +375,9 @@ private void internalPerformGetTopicsOfNamespace(long clientRequestId, String na listSizeHolder.resetIfInitializing(); writeAndFlush(Commands.newError(clientRequestId, getServerError(t), t.getMessage())); } else { - long actualSize = - r.getNonPartitionedOrPartitionTopics().stream() - .mapToInt(ByteBufUtil::utf8Bytes) // convert character count to bytes - .map(n -> n + 32) // add 32 bytes overhead for each entry - .sum(); - // update the cached size if there's a difference larger than 1 - if (Math.abs(initialSize - actualSize) > 1) { - listSizeHolder.updateSize(actualSize); - } + long actualSize = TopicListMemoryLimiter.estimateTopicListSize( + r.getNonPartitionedOrPartitionTopics()); + listSizeHolder.updateSize(actualSize); maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, isPermitRequestCancelled, permits -> { return handleWritingGetTopicsResponse(clientRequestId, r, From 5992f21c3a5702421f55a05b3aa31a3e4d4ed001 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 17 Oct 2025 17:35:13 +0300 Subject: [PATCH 45/75] Add limits for Admin API for listing topics --- .../broker/admin/impl/NamespacesBase.java | 68 ++++++++++++++++++- .../pulsar/broker/admin/v1/Namespaces.java | 2 +- .../pulsar/broker/admin/v2/Namespaces.java | 2 +- 3 files changed, 69 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index 0e821c83e610d..40ac7141e1c8b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -38,13 +38,17 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; import javax.ws.rs.WebApplicationException; import javax.ws.rs.container.AsyncResponse; +import javax.ws.rs.container.CompletionCallback; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; import javax.ws.rs.core.UriBuilder; @@ -64,6 +68,8 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; +import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -111,6 +117,8 @@ import org.apache.pulsar.common.policies.data.ValidateResult; import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -165,8 +173,66 @@ protected CompletableFuture internalCreateNamespace(Policies policies) { .thenAccept(__ -> log.info("[{}] Created namespace {}", clientAppId(), namespaceName)); } - protected CompletableFuture> internalGetListOfTopics(Policies policies, + protected CompletableFuture> internalGetListOfTopics(AsyncResponse response, Policies policies, CommandGetTopicsOfNamespace.Mode mode) { + // Use maxTopicListInFlightLimiter to limit inflight get topic listing responses + // to avoid OOME caused by a lot of clients using HTTP service lookups to list topics + AsyncDualMemoryLimiterImpl maxTopicListInFlightLimiter = + pulsar().getBrokerService().getMaxTopicListInFlightLimiter(); + TopicListSizeResultCache.ResultHolder listSizeHolder = pulsar().getBrokerService().getTopicListSizeResultCache() + .getTopicListSize(namespaceName.toString(), mode); + // setup the permit cancellation function + AtomicBoolean permitRequestCancelled = new AtomicBoolean(false); + BooleanSupplier isPermitRequestCancelled = permitRequestCancelled::get; + // add callback that releases permits when the response completes + AtomicReference initialPermitsRef = + new AtomicReference<>(); + AtomicReference permitsRef = new AtomicReference<>(); + response.register(new CompletionCallback() { + @Override + public void onComplete(Throwable throwable) { + if (throwable != null) { + // for failed request + // handle resetting the TopicListSizeResultCache.ResultHolder + listSizeHolder.resetIfInitializing(); + // cancel any pending permit request + permitRequestCancelled.set(true); + } + AsyncDualMemoryLimiter.AsyncDualMemoryLimiterPermit initialPermit = initialPermitsRef.get(); + if (initialPermit != null) { + maxTopicListInFlightLimiter.release(initialPermit); + } + AsyncDualMemoryLimiter.AsyncDualMemoryLimiterPermit permits = permitsRef.get(); + if (permits != null) { + maxTopicListInFlightLimiter.release(permits); + } + } + }); + return listSizeHolder.getSizeAsync().thenCompose(initialSize -> maxTopicListInFlightLimiter.acquire(initialSize, + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled).exceptionally(t -> { + throw new CompletionException( + new RestException(Status.TOO_MANY_REQUESTS, "Failed due to heap memory limit exceeded")); + }).thenCompose(initialPermits -> { + initialPermitsRef.set(initialPermits); + // perform the actual get list of topics operation + return doInternalGetListOfTopics(policies, mode).thenCompose(topicList -> { + long actualSize = TopicListMemoryLimiter.estimateTopicListSize(topicList); + listSizeHolder.updateSize(actualSize); + return maxTopicListInFlightLimiter.update(initialPermits, actualSize, isPermitRequestCancelled) + .exceptionally(t -> { + throw new CompletionException(new RestException(Status.TOO_MANY_REQUESTS, + "Failed due to heap memory limit exceeded")); + }).thenApply(permits -> { + permitsRef.set(permits); + initialPermitsRef.set(null); + return topicList; + }); + }); + })); + } + + private CompletableFuture> doInternalGetListOfTopics(Policies policies, + CommandGetTopicsOfNamespace.Mode mode) { switch (mode) { case ALL: return pulsar().getNamespaceService().getListOfPersistentTopics(namespaceName) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java index 4424ceded8127..ad1b223e8c2d2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java @@ -146,7 +146,7 @@ public void getTopics(@Suspended AsyncResponse response, validateNamespaceOperationAsync(NamespaceName.get(property, namespace), NamespaceOperation.GET_TOPICS) // Validate that namespace exists, throws 404 if it doesn't exist .thenCompose(__ -> getNamespacePoliciesAsync(namespaceName)) - .thenCompose(policies -> internalGetListOfTopics(policies, mode)) + .thenCompose(policies -> internalGetListOfTopics(response, policies, mode)) .thenApply(topics -> filterSystemTopic(topics, includeSystemTopic)) .thenAccept(response::resume) .exceptionally(ex -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java index 3b045259f9a71..5632de07f4709 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java @@ -131,7 +131,7 @@ public void getTopics(@Suspended AsyncResponse response, validateNamespaceOperationAsync(NamespaceName.get(tenant, namespace), NamespaceOperation.GET_TOPICS) // Validate that namespace exists, throws 404 if it doesn't exist .thenCompose(__ -> getNamespacePoliciesAsync(namespaceName)) - .thenCompose(policies -> internalGetListOfTopics(policies, mode)) + .thenCompose(policies -> internalGetListOfTopics(response, policies, mode)) .thenApply(topics -> filterSystemTopic(topics, includeSystemTopic)) .thenAccept(response::resume) .exceptionally(ex -> { From 373fe1a711735adba0b72dfa977b4894a233b60f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 17 Oct 2025 18:02:30 +0300 Subject: [PATCH 46/75] Add http lookup test --- ...essureMultipleConsumersHttpLookupTest.java | 50 +++++++++++++++++++ ...umerBackPressureMultipleConsumersTest.java | 37 +++++++++++--- 2 files changed, 79 insertions(+), 8 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersHttpLookupTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersHttpLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersHttpLookupTest.java new file mode 100644 index 0000000000000..11916926a0a5c --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersHttpLookupTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import lombok.extern.slf4j.Slf4j; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-impl") +public class PatternConsumerBackPressureMultipleConsumersHttpLookupTest + extends PatternConsumerBackPressureMultipleConsumersTest { + @Override + protected boolean useTcpLookup() { + return false; + } + + @Override + protected int getMaxRequestsInFlight() { + // AsyncHttpClient is poorly implemented and will use a single byte[] arrays for the response + // this is why higher concurrency would lead to OOME due to heap fragmentation + return 50; + } + + @Override + protected int getNumberOfRequests() { + return getNumberOfClients(); + } + + @Override + protected int getDirectMemoryRequiredMB() { + // don't do any allocations to limit direct memory + return Integer.MAX_VALUE; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java index da122718c430e..e012a3075b61d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java @@ -54,11 +54,15 @@ public class PatternConsumerBackPressureMultipleConsumersTest extends MockedPuls @Override @BeforeMethod protected void setup() throws Exception { - isTcpLookup = true; + isTcpLookup = useTcpLookup(); super.internalSetup(); setupDefaultTenantAndNamespace(); } + protected boolean useTcpLookup() { + return true; + } + @Override @AfterMethod(alwaysRun = true) protected void cleanup() throws Exception { @@ -69,15 +73,15 @@ protected void cleanup() throws Exception { public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() throws PulsarAdminException, InterruptedException, IOException { // number of requests to send to the broker - final int requests = 500; + final int requests = getNumberOfRequests(); // use multiple clients so that each client has a separate connection to the broker - final int numberOfClients = 200; + final int numberOfClients = getNumberOfClients(); // create a long topic name to consume more memory per topic final String topicName = StringUtils.repeat('a', 512) + UUID.randomUUID(); // number of topics to create final int topicCount = 8192; // maximum number of requests in flight at any given time - final int maxRequestsInFlight = 500; + final int maxRequestsInFlight = getMaxRequestsInFlight(); // create a single topic with multiple partitions admin.topics().createPartitionedTopic(topicName, topicCount); @@ -136,10 +140,12 @@ public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() } else { log.error("Failed to get topic list.", ex); } - log.info("latch-count: {}, succeed: {}, available direct mem: {} MB", latch.getCount(), - success.get(), + log.info( + "latch-count: {}, succeed: {}, available direct mem: {} MB, free heap mem: {}" + + " MB", + latch.getCount(), success.get(), (DirectMemoryUtils.jvmMaxDirectMemory() - JvmMetrics.getJvmDirectMemoryUsed()) - / (1024 * 1024)); + / (1024 * 1024), Runtime.getRuntime().freeMemory() / (1024 * 1024)); latch.countDown(); }); } catch (Exception e) { @@ -155,6 +161,18 @@ public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() validateTopiclistPrometheusMetrics(); } + protected int getNumberOfClients() { + return 200; + } + + protected int getNumberOfRequests() { + return 500; + } + + protected int getMaxRequestsInFlight() { + return 500; + } + private void validateTopiclistPrometheusMetrics() { MutableInt metricsCount = new MutableInt(0); PrometheusMetricsClient metricsClient = @@ -178,7 +196,7 @@ protected int getDirectMemoryRequiredMB() { } protected String getClientServiceUrl() { - return pulsar.getBrokerServiceUrl(); + return lookupUrl.toString(); } /** @@ -187,6 +205,9 @@ protected String getClientServiceUrl() { * @return list of ByteBufs allocated to reduce available direct memory */ private static List allocateDirectMemory(long directMemoryRequired) { + if (directMemoryRequired <= 0 || directMemoryRequired == Integer.MAX_VALUE) { + return List.of(); + } long usedMemory = JvmMetrics.getJvmDirectMemoryUsed(); long maxMemory = DirectMemoryUtils.jvmMaxDirectMemory(); long availableMemory = maxMemory - usedMemory; From 8ccd8b9a5f9fbd48e1c7a6026f6c887c5600e69f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 17 Oct 2025 22:02:28 +0300 Subject: [PATCH 47/75] Add comment about overhead --- .../pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java index 72fad7f4c8ae9..42f8d47377697 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListMemoryLimiter.java @@ -304,7 +304,9 @@ public void close() { public static long estimateTopicListSize(List topicList) { return topicList.stream() .mapToLong(ByteBufUtil::utf8Bytes) // convert character count to bytes - .map(n -> n + 32) // add 32 bytes overhead for each entry + // add 32 bytes overhead for each entry + // 16 bytes for object header, 16 bytes for java.lang.String fields + .map(n -> n + 32) .sum(); } } From 829ad5fd76d946144ee9105b9088b598d33bb640 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 27 Oct 2025 18:33:43 +0200 Subject: [PATCH 48/75] Address review comment about permits validation --- .../common/semaphore/AsyncSemaphoreImpl.java | 18 ++++---- .../semaphore/AsyncSemaphoreImplTest.java | 41 ++++++++----------- 2 files changed, 27 insertions(+), 32 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index 775e95178eb7c..d90e596cc59bc 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -100,9 +100,7 @@ public CompletableFuture acquire(long permits, BooleanSupp private CompletableFuture internalAcquire(long permits, long acquirePermits, BooleanSupplier isCancelled) { - if (!isPermitsValidForAcquiring(permits)) { - throw new IllegalArgumentException("Invalid permits value: " + permits); - } + validatePermits(permits); // if maximum permits is <= 0, then the semaphore is unbounded if (isUnbounded()) { @@ -141,8 +139,14 @@ private CompletableFuture internalAcquire(long permits, lo return future; } - private boolean isPermitsValidForAcquiring(long permits) { - return permits >= 0 && (isUnbounded() || permits <= maxPermits); + private void validatePermits(long permits) { + if (permits < 0) { + throw new IllegalArgumentException("Invalid negative permits value: " + permits); + } + if (!isUnbounded() && permits > maxPermits) { + throw new IllegalArgumentException( + "Requested permits=" + permits + " is larger than maxPermits=" + maxPermits); + } } private boolean isUnbounded() { @@ -158,9 +162,7 @@ private void recordQueueLatency(long ageNanos) { @Override public CompletableFuture update(AsyncSemaphorePermit permit, long newPermits, BooleanSupplier isCancelled) { - if (!isPermitsValidForAcquiring(newPermits)) { - throw new IllegalArgumentException("Invalid permits value: " + newPermits); - } + validatePermits(newPermits); if (isUnbounded()) { return CompletableFuture.completedFuture(new SemaphorePermit(newPermits)); } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java index 73b62a98fe30d..b409c422e36fa 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.common.semaphore; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -183,24 +184,20 @@ public void testCancellation() throws Exception { public void testInvalidPermits() { semaphore = new AsyncSemaphoreImpl(10, 10, 5000); - try { - semaphore.acquire(-1, () -> false); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid permits value")); - } + assertThatThrownBy(() -> + semaphore.acquire(-1, () -> false) + ).isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid negative permits value: -1"); } @Test public void testInvalidPermitsExceedingMaxPermits() { semaphore = new AsyncSemaphoreImpl(10, 10, 5000); - try { - semaphore.acquire(11, () -> false); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid permits value")); - } + assertThatThrownBy(() -> + semaphore.acquire(11, () -> false) + ).isInstanceOf(IllegalArgumentException.class) + .hasMessage("Requested permits=11 is larger than maxPermits=10"); } @Test @@ -284,12 +281,10 @@ public void testUpdateWithInvalidPermits() throws Exception { CompletableFuture future = semaphore.acquire(5, () -> false); AsyncSemaphorePermit permit = future.get(1, TimeUnit.SECONDS); - try { - semaphore.update(permit, -1, () -> false); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid permits value")); - } + assertThatThrownBy(() -> + semaphore.update(permit, -1, () -> false) + ).isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid negative permits value: -1"); semaphore.release(permit); } @@ -301,12 +296,10 @@ public void testUpdateWithInvalidPermitsExceedingMaxPermits() throws Exception { CompletableFuture future = semaphore.acquire(5, () -> false); AsyncSemaphorePermit permit = future.get(1, TimeUnit.SECONDS); - try { - semaphore.update(permit, 11, () -> false); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid permits value")); - } + assertThatThrownBy(() -> + semaphore.update(permit, 11, () -> false) + ).isInstanceOf(IllegalArgumentException.class) + .hasMessage("Requested permits=11 is larger than maxPermits=10"); semaphore.release(permit); } From e9ebebbe52b2d7ce2503a7c7054be2705931ccbc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 27 Oct 2025 18:36:40 +0200 Subject: [PATCH 49/75] Add comments to address review comment --- .../pulsar/broker/topiclistlimit/TopicListSizeResultCache.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java index e6951b00be93f..45f5fbf953f60 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java @@ -98,8 +98,10 @@ public void updateSize(long actualSize) { }); CompletableFuture currentFuture = topicListSizeFuture.get(); if (currentFuture != null && !currentFuture.isDone()) { + // complete the future if it's not done yet currentFuture.complete(existingSizeValue); } else if (currentFuture == null || currentFuture.join().longValue() != existingSizeValue) { + // only update the future if the current value is different from the existing value topicListSizeFuture.compareAndSet(currentFuture, CompletableFuture.completedFuture(existingSizeValue)); } } From 0ddfe52490a299df13767a058b1dcaf32b37f61e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 27 Oct 2025 18:38:49 +0200 Subject: [PATCH 50/75] Don't use CompletableFuture.join so that the logic is more clear --- .../pulsar/broker/topiclistlimit/TopicListSizeResultCache.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java index 45f5fbf953f60..9fe65abab93f2 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java @@ -100,7 +100,7 @@ public void updateSize(long actualSize) { if (currentFuture != null && !currentFuture.isDone()) { // complete the future if it's not done yet currentFuture.complete(existingSizeValue); - } else if (currentFuture == null || currentFuture.join().longValue() != existingSizeValue) { + } else if (currentFuture == null || currentFuture.getNow(0L).longValue() != existingSizeValue) { // only update the future if the current value is different from the existing value topicListSizeFuture.compareAndSet(currentFuture, CompletableFuture.completedFuture(existingSizeValue)); } From 498c442f0021d9f8f3552f670d873862bec09c95 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 30 Oct 2025 18:48:38 +0200 Subject: [PATCH 51/75] Add more tests for validating updating of permits --- .../common/semaphore/AsyncSemaphoreImplTest.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java index b409c422e36fa..51df4a76e2076 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java @@ -251,7 +251,22 @@ public void testUpdatePermitsIncrease() throws Exception { assertNotNull(permit2); assertEquals(permit2.getPermits(), 8); + CompletableFuture future3 = semaphore.acquire(2, () -> false); + AsyncSemaphorePermit permit3 = future3.get(1, TimeUnit.SECONDS); + assertNotNull(permit3); + assertEquals(permit3.getPermits(), 2); + + CompletableFuture future4 = semaphore.acquire(1, () -> false); + Thread.sleep(1000); + // no more permits available, this won't complete + assertThat(future4).isNotDone(); + + // release permits semaphore.release(permit2); + + // now future4 should complete + assertThat(future4).succeedsWithin(1, TimeUnit.SECONDS) + .satisfies(p -> assertEquals(p.getPermits(), 1)); } @Test From e5e51ea83a68d128019e26ae64597ba4eb4b0f6f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 30 Oct 2025 18:49:22 +0200 Subject: [PATCH 52/75] Refactor the code for easier readability --- .../apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index d90e596cc59bc..907e37ca3b778 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -172,13 +172,14 @@ public CompletableFuture update(AsyncSemaphorePermit permi castToImplementation(permit).releasePermits(); if (additionalPermits > 0) { return internalAcquire(newPermits, additionalPermits, isCancelled); - } else { + } + if (additionalPermits < 0) { // new permits are less than the old ones, so we return the difference availablePermits.addAndGet(-additionalPermits); processQueue(); - // return the new permits immediately - return CompletableFuture.completedFuture(new SemaphorePermit(newPermits)); } + // return the new permits immediately + return CompletableFuture.completedFuture(new SemaphorePermit(newPermits)); } @Override From 8e853b14df8fee1752b32df611c98523d266d0a2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 30 Oct 2025 18:54:22 +0200 Subject: [PATCH 53/75] No need to add permits if the permits were already released --- .../apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index 907e37ca3b778..cde224b53f6d3 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -187,8 +187,11 @@ public void release(AsyncSemaphorePermit permit) { if (isUnbounded()) { return; } - availablePermits.addAndGet(castToImplementation(permit).releasePermits()); - processQueue(); + long releasedPermits = castToImplementation(permit).releasePermits(); + if (releasedPermits > 0) { + availablePermits.addAndGet(releasedPermits); + processQueue(); + } } @Override From 2f7f3b44204386cd7549eef5612f66b0c4fc2bf8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 30 Oct 2025 18:57:42 +0200 Subject: [PATCH 54/75] Add javadoc about the usage of update --- .../java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java index 0d648d5067260..c53823a47517a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java @@ -40,6 +40,8 @@ public interface AsyncSemaphore { /** * Acquire or release permits for previously acquired permits by updating the permits. * Returns a future that completes when permits are available. + * Please notice that the permits in the original AsyncSemaphorePermit instance will be released and a new instance + * will be returned with updated permits. * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full * From d9a9e944da4e8b4f1b45e1392445c6b4bb2201b7 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 30 Oct 2025 19:02:34 +0200 Subject: [PATCH 55/75] Don't release original permits unless permits were successfully acquired --- .../common/semaphore/AsyncSemaphore.java | 4 +- .../common/semaphore/AsyncSemaphoreImpl.java | 13 +++++-- .../semaphore/AsyncSemaphoreImplTest.java | 39 +++++++++++++++++++ 3 files changed, 50 insertions(+), 6 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java index c53823a47517a..53ca0670ae631 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphore.java @@ -40,8 +40,8 @@ public interface AsyncSemaphore { /** * Acquire or release permits for previously acquired permits by updating the permits. * Returns a future that completes when permits are available. - * Please notice that the permits in the original AsyncSemaphorePermit instance will be released and a new instance - * will be returned with updated permits. + * The provided permit is released when the permits are successfully acquired and the returned updated + * permit replaces the old instance. * It will complete exceptionally with AsyncSemaphore.PermitAcquireTimeoutException on timeout * and exceptionally with AsyncSemaphore.PermitAcquireQueueFullException when queue full * diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index cde224b53f6d3..2c94ea210d3a7 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -168,16 +168,21 @@ public CompletableFuture update(AsyncSemaphorePermit permi } long oldPermits = permit.getPermits(); long additionalPermits = newPermits - oldPermits; - // mark the old permits as released without adding the permits to availablePermits - castToImplementation(permit).releasePermits(); if (additionalPermits > 0) { - return internalAcquire(newPermits, additionalPermits, isCancelled); + return internalAcquire(newPermits, additionalPermits, isCancelled) + .thenApply(p -> { + // mark the old permits as released without adding the permits to availablePermits + castToImplementation(permit).releasePermits(); + return p; + }); } if (additionalPermits < 0) { // new permits are less than the old ones, so we return the difference availablePermits.addAndGet(-additionalPermits); processQueue(); } + // mark the old permits as released without adding the permits to availablePermits + castToImplementation(permit).releasePermits(); // return the new permits immediately return CompletableFuture.completedFuture(new SemaphorePermit(newPermits)); } @@ -257,7 +262,7 @@ private synchronized void internalProcessQueue() { SemaphorePermit permit = new SemaphorePermit(request.permits); recordQueueLatency(request.getAgeNanos()); boolean futureCompleted = request.future.complete(permit); - if (!futureCompleted){ + if (!futureCompleted) { // request was already cancelled, return permits availablePermits.addAndGet(request.acquirePermits); } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java index 51df4a76e2076..7f3f66c109460 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java @@ -25,6 +25,7 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import java.time.Duration; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -599,4 +600,42 @@ public void testUnboundedSemaphoreCancellationIgnored() throws Exception { assertTrue(future.isDone()); assertFalse(future.isCompletedExceptionally()); } + + @Test + public void testPermitsArentAcquiredInUpdateWhenCancelled() throws Exception { + semaphore = new AsyncSemaphoreImpl(10, 10, 5000); + + // setup + AtomicBoolean cancelled = new AtomicBoolean(false); + // acquire 5 permits + CompletableFuture future = semaphore.acquire(5, cancelled::get); + assertThat(future).succeedsWithin(Duration.ofSeconds(1)) + .satisfies(p -> assertThat(p.getPermits()).isEqualTo(5)); + AsyncSemaphorePermit permit = future.join(); + assertThat(semaphore.getAcquiredPermits()).isEqualTo(5); + + // when permits are update when the request is already cancelled + cancelled.set(true); + CompletableFuture future2 = semaphore.update(permit, 10, cancelled::get); + assertThat(future2).failsWithin(Duration.ofSeconds(1)) + .withThrowableThat().havingRootCause() + .isInstanceOf(AsyncSemaphore.PermitAcquireCancelledException.class); + assertThat(semaphore.getAcquiredPermits()).isEqualTo(5); + + // then no permits should be acquired so that + // when a new acquisition with update is made, it should succeed + cancelled.set(false); + CompletableFuture future3 = semaphore.update(permit, 10, cancelled::get); + assertThat(future3).succeedsWithin(Duration.ofSeconds(1)) + .satisfies(p -> assertThat(p.getPermits()).isEqualTo(10)); + assertThat(semaphore.getAcquiredPermits()).isEqualTo(10); + + // when original permit is released, it shouldn't reduce acquired permits + semaphore.release(permit); + assertThat(semaphore.getAcquiredPermits()).isEqualTo(10); + + // when updated permit is released, it should reduce acquired permits + semaphore.release(future3.join()); + assertThat(semaphore.getAcquiredPermits()).isEqualTo(0); + } } \ No newline at end of file From 34a57046defa63e56caedd28a57c8ec4153ba53e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 30 Oct 2025 19:32:45 +0200 Subject: [PATCH 56/75] Add logic for handling CompletableFuture.cancel --- .../common/semaphore/AsyncSemaphoreImpl.java | 46 +++++++++++++++---- .../semaphore/AsyncSemaphoreImplTest.java | 28 +++++++++++ 2 files changed, 66 insertions(+), 8 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java index 2c94ea210d3a7..7fc506753a8b0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImpl.java @@ -21,6 +21,7 @@ import io.netty.util.concurrent.DefaultThreadFactory; import java.util.Queue; import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; @@ -32,6 +33,7 @@ import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.function.BooleanSupplier; import java.util.function.LongConsumer; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.Runnables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,6 +53,7 @@ public class AsyncSemaphoreImpl implements AsyncSemaphore, AutoCloseable { private final LongConsumer queueLatencyRecorder; private final AtomicBoolean closed = new AtomicBoolean(false); private final Runnable processQueueRunnable = Runnables.catchingAndLoggingThrowables(this::internalProcessQueue); + private final ScheduledFuture processQueueScheduledFuture; /** * Creates an AsyncSemaphoreImpl with the given parameters. @@ -86,6 +89,11 @@ public AsyncSemaphoreImpl(long maxPermits, int maxQueueSize, long timeoutMillis, this.executor = executor; this.shutdownExecutor = shutdownExecutor; this.queueLatencyRecorder = queueLatencyRecorder; + // scheduled task that runs the processQueue method every half of the timeout + // this is to support cancellation in cases where the head of the queue request is blocking others + // from proceeding and it happens to already be cancelled. + this.processQueueScheduledFuture = executor != null + ? executor.schedule(processQueueRunnable, timeoutMillis / 2, TimeUnit.MILLISECONDS) : null; } private static ScheduledExecutorService createExecutor() { @@ -169,12 +177,24 @@ public CompletableFuture update(AsyncSemaphorePermit permi long oldPermits = permit.getPermits(); long additionalPermits = newPermits - oldPermits; if (additionalPermits > 0) { - return internalAcquire(newPermits, additionalPermits, isCancelled) - .thenApply(p -> { - // mark the old permits as released without adding the permits to availablePermits - castToImplementation(permit).releasePermits(); - return p; - }); + CompletableFuture acquireFuture = + internalAcquire(newPermits, additionalPermits, isCancelled); + // return a future that completes after original permits have been released when the acquisition + // has been successfully completed + CompletableFuture returnedFuture = + acquireFuture.thenApply(p -> { + // mark the old permits as released without adding the permits to availablePermits + castToImplementation(permit).releasePermits(); + return p; + }); + // add cancellation support for returned future, so that it cancels the acquireFuture if the returnedFuture + // is cancelled + returnedFuture.whenComplete((p, t) -> { + if (t != null && FutureUtil.unwrapCompletionException(t) instanceof CancellationException) { + acquireFuture.cancel(false); + } + }); + return returnedFuture; } if (additionalPermits < 0) { // new permits are less than the old ones, so we return the difference @@ -236,7 +256,7 @@ private void processQueue() { } private synchronized void internalProcessQueue() { - while (!closed.get()) { + while (!closed.get() && !queue.isEmpty()) { long current = availablePermits.get(); if (current <= 0) { break; @@ -255,6 +275,13 @@ private synchronized void internalProcessQueue() { continue; } + // request future has been completed by user code cancellation, remove it from the queue + if (request.future.isDone()) { + request.cancelTimeoutTask(); + queue.remove(request); + continue; + } + if (request.acquirePermits <= current) { availablePermits.addAndGet(-request.acquirePermits); request.cancelTimeoutTask(); @@ -263,7 +290,7 @@ private synchronized void internalProcessQueue() { recordQueueLatency(request.getAgeNanos()); boolean futureCompleted = request.future.complete(permit); if (!futureCompleted) { - // request was already cancelled, return permits + // request was cancelled by user code, return permits availablePermits.addAndGet(request.acquirePermits); } } else { @@ -275,6 +302,9 @@ private synchronized void internalProcessQueue() { @Override public void close() { if (closed.compareAndSet(false, true)) { + if (processQueueScheduledFuture != null) { + processQueueScheduledFuture.cancel(false); + } while (!queue.isEmpty()) { PendingRequest request = queue.poll(); request.cancelTimeoutTask(); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java index 7f3f66c109460..ec2e01e1a718c 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncSemaphoreImplTest.java @@ -638,4 +638,32 @@ public void testPermitsArentAcquiredInUpdateWhenCancelled() throws Exception { semaphore.release(future3.join()); assertThat(semaphore.getAcquiredPermits()).isEqualTo(0); } + + @Test + public void testSupportCompletableFutureCancel() throws Exception { + int timeoutMillis = 200; + semaphore = new AsyncSemaphoreImpl(10, 10, timeoutMillis); + + // setup + AtomicBoolean cancelled = new AtomicBoolean(false); + // acquire 5 permits + CompletableFuture future = semaphore.acquire(5, cancelled::get); + assertThat(future).succeedsWithin(Duration.ofSeconds(1)) + .satisfies(p -> assertThat(p.getPermits()).isEqualTo(5)); + assertThat(semaphore.getAcquiredPermits()).isEqualTo(5); + + // attempt to acquire 5 permits + CompletableFuture future2 = semaphore.acquire(10, cancelled::get); + + // attempt to acquire 5 permits + CompletableFuture future3 = semaphore.acquire(5, cancelled::get); + // cancel future2 + future2.cancel(true); + assertThat(future2).isCancelled(); + // now future3 should succeed + assertThat(future3).succeedsWithin(Duration.ofSeconds(1)) + .satisfies(p -> assertThat(p.getPermits()).isEqualTo(5)); + + assertThat(semaphore.getAcquiredPermits()).isEqualTo(10); + } } \ No newline at end of file From 6e96e934d78d3edf772c7fdbcf1da882817615ba Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 30 Oct 2025 20:41:43 +0200 Subject: [PATCH 57/75] Add test case to concern about permit leak or extra permits --- .../AsyncDualMemoryLimiterUtilTest.java | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java index 8be79261f819e..13fdc2a3bafa0 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.common.semaphore; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -32,6 +33,7 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandlerContext; +import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -488,6 +490,47 @@ public void testWithPermitsFutureMultipleConcurrent() throws Exception { } } + @Test + public void testWithAcquiredAndUpdatedPermitsDoesntLeakPermitsWhenUpdatedSizeIsLarger() throws Exception { + for (int i = 0; i < 10_000; i++) { + doTestWithAcquiredPermitsAndUpdatedPermits(50, 100); + } + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAcquiredPermits()).isEqualTo(0); + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAvailablePermits()).isEqualTo(10000); + } + + @Test + public void testWithAcquiredAndUpdatedPermitsDoesntLeakPermitsWhenUpdatedSizeIsSmaller() throws Exception { + for (int i = 0; i < 10_000; i++) { + doTestWithAcquiredPermitsAndUpdatedPermits(100, 50); + } + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAcquiredPermits()).isEqualTo(0); + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAvailablePermits()).isEqualTo(10000); + } + + private void doTestWithAcquiredPermitsAndUpdatedPermits(int initialMemorySize, int newMemorySize) { + CompletableFuture result = limiter.withAcquiredPermits( + initialMemorySize, // estimated permits + LimitType.HEAP_MEMORY, + () -> false, + firstPermit -> { + // Simulate getting actual size and updating permits + return limiter.withUpdatedPermits( + firstPermit, + newMemorySize, + () -> false, + secondPermit -> { + return CompletableFuture.completedFuture(null); + }, + throwable -> CompletableFuture.failedFuture(throwable) + ); + }, + throwable -> CompletableFuture.failedFuture(throwable) + ); + assertThat(result).succeedsWithin(Duration.ofSeconds(1)); + } + + private BaseCommand createTestCommand() { BaseCommand command = new BaseCommand().setType(BaseCommand.Type.PING); command.setPing(); From 578f0272cf80decc00fdc01f0ef9656afaf3e1ce Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 4 Nov 2025 14:45:52 +0200 Subject: [PATCH 58/75] Add more test cases for leaks when there are exceptions --- .../AsyncDualMemoryLimiterUtilTest.java | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java index 13fdc2a3bafa0..6c6d919bf453b 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/semaphore/AsyncDualMemoryLimiterUtilTest.java @@ -530,6 +530,48 @@ private void doTestWithAcquiredPermitsAndUpdatedPermits(int initialMemorySize, i assertThat(result).succeedsWithin(Duration.ofSeconds(1)); } + @Test + public void testWithAcquiredPermitsDoesntLeakPermitsWhenExceptionIsThrown() throws Exception { + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAvailablePermits()).isEqualTo(10000); + CompletableFuture result = limiter.withAcquiredPermits( + 90, // estimated permits + LimitType.HEAP_MEMORY, + () -> false, + firstPermit -> { + throw new RuntimeException("Exception in withAcquiredPermits"); + }, + throwable -> CompletableFuture.failedFuture(throwable) + ); + assertThat(result).failsWithin(Duration.ofSeconds(1)); + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAcquiredPermits()).isEqualTo(0); + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAvailablePermits()).isEqualTo(10000); + } + + @Test + public void testWithAcquiredAndUpdatedPermitsDoesntLeakPermitsWhenExceptionIsThrown() throws Exception { + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAvailablePermits()).isEqualTo(10000); + CompletableFuture result = limiter.withAcquiredPermits( + 90, // estimated permits + LimitType.HEAP_MEMORY, + () -> false, + firstPermit -> { + // Simulate getting actual size and updating permits + return limiter.withUpdatedPermits( + firstPermit, + 100, + () -> false, + secondPermit -> { + throw new RuntimeException("Exception in withUpdatedPermits"); + }, + throwable -> CompletableFuture.failedFuture(throwable) + ); + }, + throwable -> CompletableFuture.failedFuture(throwable) + ); + assertThat(result).failsWithin(Duration.ofSeconds(1)); + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAcquiredPermits()).isEqualTo(0); + assertThat(limiter.getLimiter(LimitType.HEAP_MEMORY).getAvailablePermits()).isEqualTo(10000); + } private BaseCommand createTestCommand() { BaseCommand command = new BaseCommand().setType(BaseCommand.Type.PING); From c71cd3c9dfb795e15dbf6f5a9118d5dd2010c66e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Nov 2025 15:10:30 +0200 Subject: [PATCH 59/75] Retry indefinitely in TopicListService --- .../broker/service/TopicListService.java | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index d50311edccf88..c61939690e4bd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -51,8 +51,6 @@ import org.slf4j.LoggerFactory; public class TopicListService { - private static final int MAX_RETRY_COUNT = 10; - public static class TopicListWatcher implements BiConsumer { /** Topic names which are matching, the topic name contains the partition suffix. **/ @@ -235,7 +233,7 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo } private void sendTopicListSuccessWithRetries(long watcherId, long requestId, List topicList, String hash) { - performOperationWithRetries("topic list success", permitAcquireErrorHandler -> + performOperationWithRetries(watcherId, "topic list success", permitAcquireErrorHandler -> () -> connection.getCommandSender() .sendWatchTopicListSuccess(requestId, watcherId, hash, topicList, permitAcquireErrorHandler)); } @@ -245,6 +243,9 @@ private void sendTopicListSuccessWithRetries(long watcherId, long requestId, Lis */ public void initializeTopicsListWatcher(CompletableFuture watcherFuture, NamespaceName namespace, long watcherId, TopicsPattern topicsPattern) { + + // TODO: add heap limiter here + namespaceService.getListOfPersistentTopics(namespace). thenApply(topics -> { TopicListWatcher watcher = new TopicListWatcher(this, watcherId, topicsPattern, topics); @@ -310,13 +311,13 @@ public void deleteTopicListWatcher(Long watcherId) { */ public void sendTopicListUpdate(long watcherId, String topicsHash, List deletedTopics, List newTopics) { - performOperationWithRetries("topic list update", permitAcquireErrorHandler -> + performOperationWithRetries(watcherId, "topic list update", permitAcquireErrorHandler -> () -> connection.getCommandSender() .sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash, permitAcquireErrorHandler)); } // performs an operation with retries, if the operation fails, it will retry after a backoff period - private void performOperationWithRetries(String operationName, + private void performOperationWithRetries(long watcherId, String operationName, Function, Supplier>> asyncOperationFactory) { // holds a reference to the operation, this is to resolve a circular dependency between the error handler and @@ -324,11 +325,15 @@ private void performOperationWithRetries(String operationName, AtomicReference operationRef = new AtomicReference<>(); // create the error handler for the operation Consumer permitAcquireErrorHandler = - createPermitAcquireErrorHandler(operationName, operationRef); + createPermitAcquireErrorHandler(watcherId, operationName, operationRef); // create the async operation using the factory function. Pass the error handler to the factory function. Supplier> asyncOperation = asyncOperationFactory.apply(permitAcquireErrorHandler); // set the operation to run into the operation reference operationRef.set(Runnables.catchingAndLoggingThrowables(() -> { + if (!connection.isActive() || !watchers.containsKey(watcherId)) { + // do nothing if the connection has already been closed or the watcher has been removed + return; + } asyncOperation.get().thenRun(() -> retryBackoff.reset()); })); // run the operation @@ -336,7 +341,7 @@ private void performOperationWithRetries(String operationName, } // retries an operation up to MAX_RETRY_COUNT times with backoff - private Consumer createPermitAcquireErrorHandler(String operationName, + private Consumer createPermitAcquireErrorHandler(long watcherId, String operationName, AtomicReference operationRef) { ScheduledExecutorService scheduledExecutor = connection.ctx().channel().eventLoop(); AtomicInteger retryCount = new AtomicInteger(0); @@ -344,20 +349,14 @@ private Consumer createPermitAcquireErrorHandler(String operationName Throwable unwrappedException = FutureUtil.unwrapCompletionException(t); if (unwrappedException instanceof AsyncSemaphore.PermitAcquireCancelledException || unwrappedException instanceof AsyncSemaphore.PermitAcquireAlreadyClosedException - || !connection.isActive()) { + || !connection.isActive() + || !watchers.containsKey(watcherId)) { return; } - if (retryCount.incrementAndGet() < MAX_RETRY_COUNT) { - long retryDelay = retryBackoff.next(); - log.info("[{}] Cannot acquire direct memory tokens for sending {}. Retry {}/{} in {} ms. {}", - connection, operationName, retryCount.get(), MAX_RETRY_COUNT, retryDelay, - t.getMessage()); - scheduledExecutor.schedule(operationRef.get(), retryDelay, TimeUnit.MILLISECONDS); - } else { - log.warn("[{}] Cannot acquire direct memory tokens for sending {}." - + "State will be inconsistent on the client. {}", connection, operationName, - t.getMessage()); - } + long retryDelay = retryBackoff.next(); + log.info("[{}] Cannot acquire direct memory tokens for sending {}. Retry {} in {} ms. {}", connection, + operationName, retryCount.get(), retryDelay, t.getMessage()); + scheduledExecutor.schedule(operationRef.get(), retryDelay, TimeUnit.MILLISECONDS); }; } } From 587bf806a0428dfd2010f281b7810bd80e81d0f9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Nov 2025 15:20:09 +0200 Subject: [PATCH 60/75] Improve closing in TopicListService --- .../broker/service/TopicListService.java | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index c61939690e4bd..417a6955bfc4e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -99,6 +99,10 @@ public void accept(String topicName, NotificationType notificationType) { topicListService.sendTopicListUpdate(id, hash, deletedTopics, newTopics); } } + + public void close() { + + } } @@ -274,7 +278,7 @@ public void handleWatchTopicListClose(CommandWatchTopicListClose commandWatchTop } public void deleteTopicListWatcher(Long watcherId) { - CompletableFuture watcherFuture = watchers.get(watcherId); + CompletableFuture watcherFuture = watchers.remove(watcherId); if (watcherFuture == null) { log.info("[{}] TopicListWatcher was not registered on the connection: {}", watcherId, connection.toString()); @@ -288,20 +292,23 @@ public void deleteTopicListWatcher(Long watcherId) { // create operation will complete, the new watcher will be discarded. log.info("[{}] Closed watcher before its creation was completed. watcherId={}", connection.toString(), watcherId); - watchers.remove(watcherId); return; } + // deregister topic listener while avoiding race conditions + watcherFuture.whenComplete((watcher, t) -> { + if (watcher != null) { + topicResources.deregisterPersistentTopicListener(watcher); + watcher.close(); + } + }); + if (watcherFuture.isCompletedExceptionally()) { log.info("[{}] Closed watcher that already failed to be created. watcherId={}", connection.toString(), watcherId); - watchers.remove(watcherId); return; } - // Proceed with normal watcher close - topicResources.deregisterPersistentTopicListener(watcherFuture.getNow(null)); - watchers.remove(watcherId); log.info("[{}] Closed watcher, watcherId={}", connection.toString(), watcherId); } From 0eb8163e96a9f864172f8821770b3f47e9689c2a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Nov 2025 19:15:13 +0200 Subject: [PATCH 61/75] Handle topic list sending and updates in order --- .../broker/service/TopicListService.java | 77 ++++++++++++++++--- .../broker/service/TopicListWatcherTest.java | 9 ++- 2 files changed, 74 insertions(+), 12 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index 417a6955bfc4e..04148b8b34cd2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -21,7 +21,10 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.concurrent.BlockingDeque; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; @@ -59,15 +62,23 @@ public static class TopicListWatcher implements BiConsumer pendingTasks = new LinkedBlockingDeque<>(); /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). */ public TopicListWatcher(TopicListService topicListService, long id, - TopicsPattern topicsPattern, List topics) { + TopicsPattern topicsPattern, List topics, + Executor executor) { this.topicListService = topicListService; this.id = id; this.topicsPattern = topicsPattern; + this.executor = executor; this.matchingTopics = TopicList.filterTopics(topics, topicsPattern); } @@ -80,6 +91,9 @@ public List getMatchingTopics() { */ @Override public void accept(String topicName, NotificationType notificationType) { + if (closed) { + return; + } String partitionedTopicName = TopicName.get(topicName).getPartitionedTopicName(); String domainLessTopicName = TopicList.removeTopicDomainScheme(partitionedTopicName); @@ -96,12 +110,38 @@ public void accept(String topicName, NotificationType notificationType) { matchingTopics.add(topicName); } String hash = TopicList.calculateHash(matchingTopics); - topicListService.sendTopicListUpdate(id, hash, deletedTopics, newTopics); + sendTopicListUpdate(hash, deletedTopics, newTopics); } } - public void close() { + private void sendTopicListUpdate(String hash, List deletedTopics, List newTopics) { + executeTask(() -> topicListService.sendTopicListUpdate(id, hash, deletedTopics, + newTopics, this::taskFinished)); + } + private synchronized void executeTask(Runnable task) { + if (closed) { + return; + } + if (!taskExecuting) { + taskExecuting = true; + executor.execute(task); + } else { + pendingTasks.add(task); + } + } + + private synchronized void taskFinished() { + Runnable task = pendingTasks.poll(); + if (task != null) { + executor.execute(task); + } else { + taskExecuting = false; + } + } + + public void close() { + closed = true; } } @@ -221,7 +261,7 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo "[{}] Received WatchTopicList for namespace [//{}] by {}", connection.toString(), namespaceName, requestId); } - sendTopicListSuccessWithRetries(watcherId, requestId, topicList, hash); + sendTopicListSuccessWithRetries(watcherId, requestId, topicList, hash, watcher::taskFinished); lookupSemaphore.release(); }) .exceptionally(ex -> { @@ -236,10 +276,19 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo }); } - private void sendTopicListSuccessWithRetries(long watcherId, long requestId, List topicList, String hash) { + private void sendTopicListSuccessWithRetries(long watcherId, long requestId, List topicList, String hash, + Runnable completionCallback) { performOperationWithRetries(watcherId, "topic list success", permitAcquireErrorHandler -> () -> connection.getCommandSender() - .sendWatchTopicListSuccess(requestId, watcherId, hash, topicList, permitAcquireErrorHandler)); + .sendWatchTopicListSuccess(requestId, watcherId, hash, topicList, permitAcquireErrorHandler) + .whenComplete((__, t) -> { + if (t != null) { + // this is an unexpected case + log.warn("[{}] Failed to send topic list success for watcherId={}. Watcher will be in " + + "inconsistent state.", connection, watcherId, t); + } + completionCallback.run(); + })); } /*** @@ -252,7 +301,8 @@ public void initializeTopicsListWatcher(CompletableFuture watc namespaceService.getListOfPersistentTopics(namespace). thenApply(topics -> { - TopicListWatcher watcher = new TopicListWatcher(this, watcherId, topicsPattern, topics); + TopicListWatcher watcher = new TopicListWatcher(this, watcherId, topicsPattern, + topics, connection.ctx().executor()); topicResources.registerPersistentTopicListener(namespace, watcher); return watcher; }). @@ -317,10 +367,19 @@ public void deleteTopicListWatcher(Long watcherId) { * @param newTopics topics names added(contains the partition suffix). */ public void sendTopicListUpdate(long watcherId, String topicsHash, List deletedTopics, - List newTopics) { + List newTopics, Runnable completionCallback) { performOperationWithRetries(watcherId, "topic list update", permitAcquireErrorHandler -> () -> connection.getCommandSender() - .sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash, permitAcquireErrorHandler)); + .sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash, + permitAcquireErrorHandler) + .whenComplete((__, t) -> { + if (t != null) { + // this is an unexpected case + log.warn("[{}] Failed to send topic list update for watcherId={}. Watcher will be in " + + "inconsistent state.", connection, watcherId, t); + } + completionCallback.run(); + })); } // performs an operation with retries, if the operation fails, it will retry after a backoff period diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java index 884cdc0ef9266..cf0db063834fa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java @@ -18,9 +18,11 @@ */ package org.apache.pulsar.broker.service; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; +import com.google.common.util.concurrent.MoreExecutors; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -53,7 +55,8 @@ public class TopicListWatcherTest { @BeforeMethod(alwaysRun = true) public void setup() { topicListService = mock(TopicListService.class); - watcher = new TopicListService.TopicListWatcher(topicListService, ID, PATTERN, INITIAL_TOPIC_LIST); + watcher = new TopicListService.TopicListWatcher(topicListService, ID, PATTERN, INITIAL_TOPIC_LIST, + MoreExecutors.directExecutor()); } @Test @@ -72,7 +75,7 @@ public void testAcceptSendsNotificationAndRemembersTopic() { "persistent://tenant/ns/topic1", "persistent://tenant/ns/topic2", newTopic); String hash = TopicList.calculateHash(allMatchingTopics); verify(topicListService).sendTopicListUpdate(ID, hash, Collections.emptyList(), - Collections.singletonList(newTopic)); + Collections.singletonList(newTopic), any()); Assert.assertEquals( allMatchingTopics, watcher.getMatchingTopics()); @@ -86,7 +89,7 @@ public void testAcceptSendsNotificationAndForgetsTopic() { List allMatchingTopics = Collections.singletonList("persistent://tenant/ns/topic2"); String hash = TopicList.calculateHash(allMatchingTopics); verify(topicListService).sendTopicListUpdate(ID, hash, - Collections.singletonList(deletedTopic), Collections.emptyList()); + Collections.singletonList(deletedTopic), Collections.emptyList(), any()); Assert.assertEquals( allMatchingTopics, watcher.getMatchingTopics()); From 508c99b60e890c26be97b8fd5f28246a94d8adce Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Nov 2025 19:18:13 +0200 Subject: [PATCH 62/75] Improve method name and comment --- .../pulsar/broker/service/TopicListService.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index 04148b8b34cd2..a9d8a8d5af542 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -278,7 +278,7 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo private void sendTopicListSuccessWithRetries(long watcherId, long requestId, List topicList, String hash, Runnable completionCallback) { - performOperationWithRetries(watcherId, "topic list success", permitAcquireErrorHandler -> + performOperationWithPermitAcquiringRetries(watcherId, "topic list success", permitAcquireErrorHandler -> () -> connection.getCommandSender() .sendWatchTopicListSuccess(requestId, watcherId, hash, topicList, permitAcquireErrorHandler) .whenComplete((__, t) -> { @@ -368,7 +368,7 @@ public void deleteTopicListWatcher(Long watcherId) { */ public void sendTopicListUpdate(long watcherId, String topicsHash, List deletedTopics, List newTopics, Runnable completionCallback) { - performOperationWithRetries(watcherId, "topic list update", permitAcquireErrorHandler -> + performOperationWithPermitAcquiringRetries(watcherId, "topic list update", permitAcquireErrorHandler -> () -> connection.getCommandSender() .sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash, permitAcquireErrorHandler) @@ -382,10 +382,12 @@ public void sendTopicListUpdate(long watcherId, String topicsHash, List })); } - // performs an operation with retries, if the operation fails, it will retry after a backoff period - private void performOperationWithRetries(long watcherId, String operationName, - Function, Supplier>> - asyncOperationFactory) { + // performs an operation with infinite permit acquiring retries. + // If acquiring permits fails, it will retry after a backoff period + private void performOperationWithPermitAcquiringRetries(long watcherId, String operationName, + Function, + Supplier>> + asyncOperationFactory) { // holds a reference to the operation, this is to resolve a circular dependency between the error handler and // the actual operation AtomicReference operationRef = new AtomicReference<>(); From 380a8e3c0a8baea2d196b2b05ac6d1625d129b48 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Nov 2025 20:09:43 +0200 Subject: [PATCH 63/75] Add heap limit handling to listing topics --- .../broker/service/TopicListService.java | 80 ++++++++++++++----- 1 file changed, 58 insertions(+), 22 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index a9d8a8d5af542..cfa59570d9a1f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -31,16 +31,21 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; +import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.TopicResources; +import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; +import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.api.proto.CommandWatchTopicListClose; import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; import org.apache.pulsar.common.semaphore.AsyncSemaphore; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.topics.TopicsPattern; @@ -150,6 +155,7 @@ public void close() { private final NamespaceService namespaceService; private final TopicResources topicResources; + private final PulsarService pulsar; private final ServerCnx connection; private final boolean enableSubscriptionPatternEvaluation; private final int maxSubscriptionPatternLength; @@ -160,6 +166,7 @@ public void close() { public TopicListService(PulsarService pulsar, ServerCnx connection, boolean enableSubscriptionPatternEvaluation, int maxSubscriptionPatternLength) { this.namespaceService = pulsar.getNamespaceService(); + this.pulsar = pulsar; this.connection = connection; this.enableSubscriptionPatternEvaluation = enableSubscriptionPatternEvaluation; this.maxSubscriptionPatternLength = maxSubscriptionPatternLength; @@ -296,30 +303,59 @@ private void sendTopicListSuccessWithRetries(long watcherId, long requestId, Lis */ public void initializeTopicsListWatcher(CompletableFuture watcherFuture, NamespaceName namespace, long watcherId, TopicsPattern topicsPattern) { - - // TODO: add heap limiter here - - namespaceService.getListOfPersistentTopics(namespace). - thenApply(topics -> { - TopicListWatcher watcher = new TopicListWatcher(this, watcherId, topicsPattern, - topics, connection.ctx().executor()); - topicResources.registerPersistentTopicListener(namespace, watcher); - return watcher; - }). - whenComplete((watcher, exception) -> { - if (exception != null) { - watcherFuture.completeExceptionally(exception); - } else { - if (!watcherFuture.complete(watcher)) { - log.warn("[{}] Watcher future was already completed. Deregistering watcherId={}.", - connection.toString(), watcherId); - topicResources.deregisterPersistentTopicListener(watcher); - } - } - }); + BooleanSupplier isPermitRequestCancelled = () -> !connection.isActive() || !watchers.containsKey(watcherId); + if (isPermitRequestCancelled.getAsBoolean()) { + return; + } + TopicListSizeResultCache.ResultHolder listSizeHolder = pulsar.getBrokerService().getTopicListSizeResultCache() + .getTopicListSize(namespace.toString(), CommandGetTopicsOfNamespace.Mode.PERSISTENT); + AsyncDualMemoryLimiter maxTopicListInFlightLimiter = pulsar.getBrokerService().getMaxTopicListInFlightLimiter(); + + listSizeHolder.getSizeAsync().thenCompose(initialSize -> { + // use heap size limiter to avoid broker getting overwhelmed by a lot of concurrent topic list requests + return maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { + return namespaceService.getListOfPersistentTopics(namespace).thenCompose(topics -> { + long actualSize = TopicListMemoryLimiter.estimateTopicListSize(topics); + listSizeHolder.updateSize(actualSize); + return maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, + isPermitRequestCancelled, updatedPermits -> { + TopicListWatcher watcher = + new TopicListWatcher(this, watcherId, topicsPattern, topics, + connection.ctx().executor()); + topicResources.registerPersistentTopicListener(namespace, watcher); + return CompletableFuture.completedFuture(watcher); + }, CompletableFuture::failedFuture); + }).whenComplete((watcher, exception) -> { + if (exception != null) { + watcherFuture.completeExceptionally(exception); + } else { + if (!watcherFuture.complete(watcher)) { + log.warn("[{}] Watcher future was already completed. Deregistering " + + "watcherId={}.", connection, watcherId); + topicResources.deregisterPersistentTopicListener(watcher); + } + } + }); + }, CompletableFuture::failedFuture); + }).exceptionally(t -> { + Throwable unwrappedException = FutureUtil.unwrapCompletionException(t); + if (!isPermitRequestCancelled.getAsBoolean() && ( + unwrappedException instanceof AsyncSemaphore.PermitAcquireTimeoutException + || unwrappedException instanceof AsyncSemaphore.PermitAcquireQueueFullException)) { + // retry with backoff if permit acquisition fails due to timeout or queue full + connection.ctx().executor() + .schedule(() -> initializeTopicsListWatcher(watcherFuture, namespace, watcherId, topicsPattern), + retryBackoff.next(), TimeUnit.MILLISECONDS); + } else { + log.warn("[{}] Failed to initialize topic list watcher watcherId={} for namespace {}.", connection, + watcherId, namespace, unwrappedException); + watcherFuture.completeExceptionally(unwrappedException); + } + return null; + }); } - public void handleWatchTopicListClose(CommandWatchTopicListClose commandWatchTopicListClose) { long requestId = commandWatchTopicListClose.getRequestId(); long watcherId = commandWatchTopicListClose.getWatcherId(); From ffe72b5a5b16f09b7ded2f5839ec4d275c475e07 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Nov 2025 20:13:11 +0200 Subject: [PATCH 64/75] Update comment --- .../java/org/apache/pulsar/broker/service/TopicListService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index cfa59570d9a1f..b56a4c5d7b615 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -444,7 +444,7 @@ private void performOperationWithPermitAcquiringRetries(long watcherId, String o operationRef.get().run(); } - // retries an operation up to MAX_RETRY_COUNT times with backoff + // retries acquiring permits until the connection is closed or the watcher is removed private Consumer createPermitAcquireErrorHandler(long watcherId, String operationName, AtomicReference operationRef) { ScheduledExecutorService scheduledExecutor = connection.ctx().channel().eventLoop(); From a3190cc5c914fefa9cdddfa7a56554723ff269f5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Nov 2025 20:37:52 +0200 Subject: [PATCH 65/75] Fix tests --- .../broker/service/TopicListService.java | 6 +++ .../broker/service/TopicListServiceTest.java | 44 +++++++++++++++++-- 2 files changed, 46 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index b56a4c5d7b615..887bd6bdf0681 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import com.google.common.annotations.VisibleForTesting; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -463,4 +464,9 @@ private Consumer createPermitAcquireErrorHandler(long watcherId, Stri scheduledExecutor.schedule(operationRef.get(), retryDelay, TimeUnit.MILLISECONDS); }; } + + @VisibleForTesting + CompletableFuture getWatcherFuture(long watcherId) { + return watchers.get(watcherId); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java index 3c12f03d2406b..4eaff70ed45f9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java @@ -18,13 +18,16 @@ */ package org.apache.pulsar.broker.service; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; @@ -32,8 +35,11 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.EventLoop; +import io.netty.util.concurrent.EventExecutor; +import io.netty.util.concurrent.ImmediateEventExecutor; import io.netty.util.concurrent.ScheduledFuture; import java.net.InetSocketAddress; +import java.time.Duration; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -47,9 +53,11 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.resources.TopicResources; +import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; import org.apache.pulsar.common.api.proto.CommandWatchTopicListClose; import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.semaphore.AsyncSemaphore; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.topics.TopicsPattern; @@ -57,6 +65,7 @@ import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; import org.testng.Assert; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -72,6 +81,7 @@ public class TopicListServiceTest { private EventLoop eventLoop; private PulsarCommandSender pulsarCommandSender; private Consumer notificationConsumer; + private AsyncDualMemoryLimiterImpl memoryLimiter; @BeforeMethod(alwaysRun = true) public void setup() throws Exception { @@ -94,6 +104,17 @@ public void setup() throws Exception { when(pulsar.getPulsarResources().getTopicResources()).thenReturn(topicResources); when(pulsar.getNamespaceService().getListOfPersistentTopics(any())).thenReturn(topicListFuture); + BrokerService brokerService = mock(BrokerService.class); + when(pulsar.getBrokerService()).thenReturn(brokerService); + TopicListSizeResultCache topicListSizeResultCache = mock(TopicListSizeResultCache.class); + when(brokerService.getTopicListSizeResultCache()).thenReturn(topicListSizeResultCache); + TopicListSizeResultCache.ResultHolder resultHolder = mock(TopicListSizeResultCache.ResultHolder.class); + doReturn(resultHolder).when(topicListSizeResultCache).getTopicListSize(anyString(), any()); + doReturn(CompletableFuture.completedFuture(1L)).when(resultHolder).getSizeAsync(); + + memoryLimiter = new AsyncDualMemoryLimiterImpl(1_000_000, 10000, 500, 1_000_000, 10000, 500); + doReturn(memoryLimiter).when(brokerService).getMaxTopicListInFlightLimiter(); + connection = mock(ServerCnx.class); when(connection.getRemoteAddress()).thenReturn(new InetSocketAddress(10000)); pulsarCommandSender = mock(PulsarCommandSender.class); @@ -107,6 +128,8 @@ public void setup() throws Exception { ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); when(connection.ctx()).thenReturn(ctx); + EventExecutor executor = ImmediateEventExecutor.INSTANCE; + doReturn(executor).when(ctx).executor(); Channel channel = mock(Channel.class); when(ctx.channel()).thenReturn(channel); eventLoop = mock(EventLoop.class); @@ -116,8 +139,15 @@ public void setup() throws Exception { } + @AfterMethod(alwaysRun = true) + void cleanup() { + if (memoryLimiter != null) { + memoryLimiter.close(); + } + } + @Test - public void testCommandWatchSuccessResponse() { + public void testCommandWatchSuccessResponse() throws InterruptedException { topicListService.handleWatchTopicList( NamespaceName.get("tenant/ns"), @@ -129,6 +159,7 @@ public void testCommandWatchSuccessResponse() { List topics = Collections.singletonList("persistent://tenant/ns/topic1"); String hash = TopicList.calculateHash(topics); topicListFuture.complete(topics); + Thread.sleep(500); Assert.assertEquals(1, lookupSemaphore.availablePermits()); verify(topicResources).registerPersistentTopicListener( eq(NamespaceName.get("tenant/ns")), any(TopicListService.TopicListWatcher.class)); @@ -162,11 +193,13 @@ public void testCommandWatchTopicListCloseRemovesListener() { lookupSemaphore); List topics = Collections.singletonList("persistent://tenant/ns/topic1"); topicListFuture.complete(topics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(1)); CommandWatchTopicListClose watchTopicListClose = new CommandWatchTopicListClose() .setRequestId(8) .setWatcherId(13); topicListService.handleWatchTopicListClose(watchTopicListClose); + verify(topicResources).deregisterPersistentTopicListener(any(TopicListService.TopicListWatcher.class)); } @@ -199,6 +232,7 @@ public void testCommandWatchSuccessRetries() { } }).when(pulsarCommandSender).sendWatchTopicListSuccess(anyLong(), anyLong(), anyString(), any(), any()); topicListFuture.complete(topics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(1)); verify(connection.getCommandSender(), times(3)) .sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(topics), any()); } @@ -214,18 +248,19 @@ public void testCommandWatchUpdate() { lookupSemaphore); List topics = Collections.singletonList("persistent://tenant/ns/topic1"); topicListFuture.complete(topics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(1)); List newTopics = Collections.singletonList("persistent://tenant/ns/topic2"); String hash = TopicList.calculateHash(ListUtils.union(topics, newTopics)); notificationConsumer.accept( new Notification(NotificationType.Created, "/managed-ledgers/tenant/ns/persistent/topic2")); - verify(connection.getCommandSender()) + verify(connection.getCommandSender(), timeout(1000L)) .sendWatchTopicListUpdate(eq(13L), eq(newTopics), any(), eq(hash), any()); hash = TopicList.calculateHash(newTopics); notificationConsumer.accept( new Notification(NotificationType.Deleted, "/managed-ledgers/tenant/ns/persistent/topic1")); - verify(connection.getCommandSender()) + verify(connection.getCommandSender(), timeout(1000L)) .sendWatchTopicListUpdate(eq(13L), eq(List.of()), eq(topics), eq(hash), any()); } @@ -240,6 +275,7 @@ public void testCommandWatchUpdateRetries() { lookupSemaphore); List topics = Collections.singletonList("persistent://tenant/ns/topic1"); topicListFuture.complete(topics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(1)); List newTopics = Collections.singletonList("persistent://tenant/ns/topic2"); String hash = TopicList.calculateHash(ListUtils.union(topics, newTopics)); @@ -262,7 +298,7 @@ public void testCommandWatchUpdateRetries() { }).when(pulsarCommandSender).sendWatchTopicListUpdate(anyLong(), any(), any(), anyString(), any()); notificationConsumer.accept( new Notification(NotificationType.Created, "/managed-ledgers/tenant/ns/persistent/topic2")); - verify(connection.getCommandSender(), times(3)) + verify(connection.getCommandSender(), timeout(1000L).times(3)) .sendWatchTopicListUpdate(eq(13L), eq(newTopics), eq(List.of()), eq(hash), any()); } } From a73d04d8e75f5427c99765e36322a64ad9c744af Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Nov 2025 14:42:29 +0200 Subject: [PATCH 66/75] Fix tests --- .../apache/pulsar/broker/service/TopicListService.java | 3 ++- .../pulsar/broker/service/TopicListWatcherTest.java | 10 ++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index 887bd6bdf0681..c6c684922fdc3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -137,7 +137,8 @@ private synchronized void executeTask(Runnable task) { } } - private synchronized void taskFinished() { + @VisibleForTesting + synchronized void taskFinished() { Runnable task = pendingTasks.poll(); if (task != null) { executor.execute(task); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java index cf0db063834fa..e9cfd432c3a85 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; @@ -57,6 +58,7 @@ public void setup() { topicListService = mock(TopicListService.class); watcher = new TopicListService.TopicListWatcher(topicListService, ID, PATTERN, INITIAL_TOPIC_LIST, MoreExecutors.directExecutor()); + watcher.taskFinished(); } @Test @@ -74,8 +76,8 @@ public void testAcceptSendsNotificationAndRemembersTopic() { List allMatchingTopics = Arrays.asList( "persistent://tenant/ns/topic1", "persistent://tenant/ns/topic2", newTopic); String hash = TopicList.calculateHash(allMatchingTopics); - verify(topicListService).sendTopicListUpdate(ID, hash, Collections.emptyList(), - Collections.singletonList(newTopic), any()); + verify(topicListService).sendTopicListUpdate(eq(ID), eq(hash), eq(Collections.emptyList()), + eq(Collections.singletonList(newTopic)), any()); Assert.assertEquals( allMatchingTopics, watcher.getMatchingTopics()); @@ -88,8 +90,8 @@ public void testAcceptSendsNotificationAndForgetsTopic() { List allMatchingTopics = Collections.singletonList("persistent://tenant/ns/topic2"); String hash = TopicList.calculateHash(allMatchingTopics); - verify(topicListService).sendTopicListUpdate(ID, hash, - Collections.singletonList(deletedTopic), Collections.emptyList(), any()); + verify(topicListService).sendTopicListUpdate(eq(ID), eq(hash), + eq(Collections.singletonList(deletedTopic)), eq(Collections.emptyList()), any()); Assert.assertEquals( allMatchingTopics, watcher.getMatchingTopics()); From d209d54a5b2eec5b1cea31c240f55a5dacd60f92 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Nov 2025 15:01:44 +0200 Subject: [PATCH 67/75] Refactor --- .../broker/service/TopicListService.java | 52 ++++++++++--------- .../broker/service/TopicListWatcherTest.java | 7 ++- 2 files changed, 30 insertions(+), 29 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index c6c684922fdc3..b9f05553446fa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -70,10 +70,8 @@ public static class TopicListWatcher implements BiConsumer pendingTasks = new LinkedBlockingDeque<>(); + private boolean sendTopicListSuccessCompleted = false; + private BlockingDeque sendTopicListUpdateTasksBeforeInit = new LinkedBlockingDeque<>(); /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). @@ -120,35 +118,36 @@ public void accept(String topicName, NotificationType notificationType) { } } - private void sendTopicListUpdate(String hash, List deletedTopics, List newTopics) { - executeTask(() -> topicListService.sendTopicListUpdate(id, hash, deletedTopics, - newTopics, this::taskFinished)); - } - - private synchronized void executeTask(Runnable task) { + private synchronized void sendTopicListUpdate(String hash, List deletedTopics, List newTopics) { if (closed) { return; } - if (!taskExecuting) { - taskExecuting = true; + Runnable task = () -> topicListService.sendTopicListUpdate(id, hash, deletedTopics, newTopics); + if (sendTopicListSuccessCompleted) { executor.execute(task); } else { - pendingTasks.add(task); + // if sendTopicListSuccess hasn't completed, add to a queue to be executed after it completes + sendTopicListUpdateTasksBeforeInit.add(task); } } @VisibleForTesting - synchronized void taskFinished() { - Runnable task = pendingTasks.poll(); - if (task != null) { + synchronized void sendTopicListSuccessCompleted() { + if (closed) { + sendTopicListUpdateTasksBeforeInit.clear(); + return; + } + // Drain all pending sendTopicListUpdate tasks + Runnable task; + while ((task = sendTopicListUpdateTasksBeforeInit.poll()) != null) { executor.execute(task); - } else { - taskExecuting = false; } + sendTopicListSuccessCompleted = true; } - public void close() { + public synchronized void close() { closed = true; + sendTopicListUpdateTasksBeforeInit.clear(); } } @@ -270,7 +269,8 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo "[{}] Received WatchTopicList for namespace [//{}] by {}", connection.toString(), namespaceName, requestId); } - sendTopicListSuccessWithRetries(watcherId, requestId, topicList, hash, watcher::taskFinished); + sendTopicListSuccessWithPermitAcquiringRetries(watcherId, requestId, topicList, hash, + watcher::sendTopicListSuccessCompleted); lookupSemaphore.release(); }) .exceptionally(ex -> { @@ -285,8 +285,9 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo }); } - private void sendTopicListSuccessWithRetries(long watcherId, long requestId, List topicList, String hash, - Runnable completionCallback) { + private void sendTopicListSuccessWithPermitAcquiringRetries(long watcherId, long requestId, List topicList, + String hash, + Runnable successfulCompletionCallback) { performOperationWithPermitAcquiringRetries(watcherId, "topic list success", permitAcquireErrorHandler -> () -> connection.getCommandSender() .sendWatchTopicListSuccess(requestId, watcherId, hash, topicList, permitAcquireErrorHandler) @@ -295,8 +296,10 @@ private void sendTopicListSuccessWithRetries(long watcherId, long requestId, Lis // this is an unexpected case log.warn("[{}] Failed to send topic list success for watcherId={}. Watcher will be in " + "inconsistent state.", connection, watcherId, t); + } else { + // completed successfully, run the callback + successfulCompletionCallback.run(); } - completionCallback.run(); })); } @@ -405,7 +408,7 @@ public void deleteTopicListWatcher(Long watcherId) { * @param newTopics topics names added(contains the partition suffix). */ public void sendTopicListUpdate(long watcherId, String topicsHash, List deletedTopics, - List newTopics, Runnable completionCallback) { + List newTopics) { performOperationWithPermitAcquiringRetries(watcherId, "topic list update", permitAcquireErrorHandler -> () -> connection.getCommandSender() .sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash, @@ -416,7 +419,6 @@ public void sendTopicListUpdate(long watcherId, String topicsHash, List log.warn("[{}] Failed to send topic list update for watcherId={}. Watcher will be in " + "inconsistent state.", connection, watcherId, t); } - completionCallback.run(); })); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java index e9cfd432c3a85..92dddd785a5f8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -58,7 +57,7 @@ public void setup() { topicListService = mock(TopicListService.class); watcher = new TopicListService.TopicListWatcher(topicListService, ID, PATTERN, INITIAL_TOPIC_LIST, MoreExecutors.directExecutor()); - watcher.taskFinished(); + watcher.sendTopicListSuccessCompleted(); } @Test @@ -77,7 +76,7 @@ public void testAcceptSendsNotificationAndRemembersTopic() { "persistent://tenant/ns/topic1", "persistent://tenant/ns/topic2", newTopic); String hash = TopicList.calculateHash(allMatchingTopics); verify(topicListService).sendTopicListUpdate(eq(ID), eq(hash), eq(Collections.emptyList()), - eq(Collections.singletonList(newTopic)), any()); + eq(Collections.singletonList(newTopic))); Assert.assertEquals( allMatchingTopics, watcher.getMatchingTopics()); @@ -91,7 +90,7 @@ public void testAcceptSendsNotificationAndForgetsTopic() { List allMatchingTopics = Collections.singletonList("persistent://tenant/ns/topic2"); String hash = TopicList.calculateHash(allMatchingTopics); verify(topicListService).sendTopicListUpdate(eq(ID), eq(hash), - eq(Collections.singletonList(deletedTopic)), eq(Collections.emptyList()), any()); + eq(Collections.singletonList(deletedTopic)), eq(Collections.emptyList())); Assert.assertEquals( allMatchingTopics, watcher.getMatchingTopics()); From baec893f22dc96dda02570b27a709104e6bea6dc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Nov 2025 17:47:48 +0200 Subject: [PATCH 68/75] Use Awaitility in the test --- .../pulsar/broker/service/TopicListServiceTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java index 4eaff70ed45f9..88ee6ffbdbd8c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java @@ -64,6 +64,7 @@ import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; +import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -147,7 +148,7 @@ void cleanup() { } @Test - public void testCommandWatchSuccessResponse() throws InterruptedException { + public void testCommandWatchSuccessResponse() { topicListService.handleWatchTopicList( NamespaceName.get("tenant/ns"), @@ -159,8 +160,7 @@ public void testCommandWatchSuccessResponse() throws InterruptedException { List topics = Collections.singletonList("persistent://tenant/ns/topic1"); String hash = TopicList.calculateHash(topics); topicListFuture.complete(topics); - Thread.sleep(500); - Assert.assertEquals(1, lookupSemaphore.availablePermits()); + Awaitility.await().untilAsserted(() -> Assert.assertEquals(1, lookupSemaphore.availablePermits())); verify(topicResources).registerPersistentTopicListener( eq(NamespaceName.get("tenant/ns")), any(TopicListService.TopicListWatcher.class)); verify(connection.getCommandSender()).sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(topics), any()); @@ -176,7 +176,7 @@ public void testCommandWatchErrorResponse() { topicsPatternImplementation, null, lookupSemaphore); topicListFuture.completeExceptionally(new PulsarServerException("Error")); - Assert.assertEquals(1, lookupSemaphore.availablePermits()); + Awaitility.await().untilAsserted(() -> Assert.assertEquals(1, lookupSemaphore.availablePermits())); verifyNoInteractions(topicResources); verify(connection.getCommandSender()).sendErrorResponse(eq(7L), any(ServerError.class), eq(PulsarServerException.class.getCanonicalName() + ": Error")); From 32d28fbfeea6a189da877bcc3a93f649a5ecdc0b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Nov 2025 18:16:54 +0200 Subject: [PATCH 69/75] Fix retrying in initializeTopicsListWatcher --- .../broker/service/TopicListService.java | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index b9f05553446fa..af4836610d06c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -320,24 +320,35 @@ public void initializeTopicsListWatcher(CompletableFuture watc // use heap size limiter to avoid broker getting overwhelmed by a lot of concurrent topic list requests return maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { + AtomicReference watcherRef = new AtomicReference<>(); return namespaceService.getListOfPersistentTopics(namespace).thenCompose(topics -> { long actualSize = TopicListMemoryLimiter.estimateTopicListSize(topics); listSizeHolder.updateSize(actualSize); + // register watcher immediately so that we don't lose events + TopicListWatcher watcher = + new TopicListWatcher(this, watcherId, topicsPattern, topics, + connection.ctx().executor()); + watcherRef.set(watcher); + topicResources.registerPersistentTopicListener(namespace, watcher); + // use updated permits to slow down responses so that backpressure gets applied return maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, isPermitRequestCancelled, updatedPermits -> { - TopicListWatcher watcher = - new TopicListWatcher(this, watcherId, topicsPattern, topics, - connection.ctx().executor()); - topicResources.registerPersistentTopicListener(namespace, watcher); + // just return the watcher which was already created before return CompletableFuture.completedFuture(watcher); }, CompletableFuture::failedFuture); }).whenComplete((watcher, exception) -> { if (exception != null) { - watcherFuture.completeExceptionally(exception); + if (watcherRef.get() != null) { + watcher.close(); + topicResources.deregisterPersistentTopicListener(watcherRef.get()); + } + // triggers a retry + throw FutureUtil.wrapToCompletionException(exception); } else { if (!watcherFuture.complete(watcher)) { log.warn("[{}] Watcher future was already completed. Deregistering " + "watcherId={}.", connection, watcherId); + watcher.close(); topicResources.deregisterPersistentTopicListener(watcher); } } From ca665addcffdcc190e410b8d78cfe583ee89a58b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Nov 2025 21:01:52 +0200 Subject: [PATCH 70/75] Test permit acquiring retries --- .../broker/service/TopicListService.java | 8 ++- .../broker/service/TopicListServiceTest.java | 61 ++++++++++++++----- 2 files changed, 54 insertions(+), 15 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index af4836610d06c..f8eb5624e6f0e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -333,6 +333,8 @@ public void initializeTopicsListWatcher(CompletableFuture watc // use updated permits to slow down responses so that backpressure gets applied return maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, isPermitRequestCancelled, updatedPermits -> { + // reset retry backoff + retryBackoff.reset(); // just return the watcher which was already created before return CompletableFuture.completedFuture(watcher); }, CompletableFuture::failedFuture); @@ -360,9 +362,13 @@ public void initializeTopicsListWatcher(CompletableFuture watc unwrappedException instanceof AsyncSemaphore.PermitAcquireTimeoutException || unwrappedException instanceof AsyncSemaphore.PermitAcquireQueueFullException)) { // retry with backoff if permit acquisition fails due to timeout or queue full + long retryAfterMillis = this.retryBackoff.next(); + log.info("[{}] {} when initializing topic list watcher watcherId={} for namespace {}. Retrying in {} " + + "ms.", connection, unwrappedException.getMessage(), watcherId, namespace, + retryAfterMillis); connection.ctx().executor() .schedule(() -> initializeTopicsListWatcher(watcherFuture, namespace, watcherId, topicsPattern), - retryBackoff.next(), TimeUnit.MILLISECONDS); + retryAfterMillis, TimeUnit.MILLISECONDS); } else { log.warn("[{}] Failed to initialize topic list watcher watcherId={} for namespace {}.", connection, watcherId, namespace, unwrappedException); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java index 88ee6ffbdbd8c..6c8cebd0016d6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java @@ -43,10 +43,12 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.collections4.ListUtils; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -57,6 +59,7 @@ import org.apache.pulsar.common.api.proto.CommandWatchTopicListClose; import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.semaphore.AsyncSemaphore; import org.apache.pulsar.common.topics.TopicList; @@ -70,6 +73,7 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +@Slf4j public class TopicListServiceTest { private TopicListService topicListService; @@ -129,12 +133,26 @@ public void setup() throws Exception { ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); when(connection.ctx()).thenReturn(ctx); - EventExecutor executor = ImmediateEventExecutor.INSTANCE; + EventExecutor executor = spy(ImmediateEventExecutor.INSTANCE); doReturn(executor).when(ctx).executor(); + doAnswer(invocationOnMock -> { + Runnable runnable = invocationOnMock.getArgument(0); + // run immediately + log.info("Running runnable immediately"); + runnable.run(); + return mock(ScheduledFuture.class); + }).when(executor).schedule(any(Runnable.class), anyLong(), any()); Channel channel = mock(Channel.class); when(ctx.channel()).thenReturn(channel); eventLoop = mock(EventLoop.class); when(channel.eventLoop()).thenReturn(eventLoop); + doAnswer(invocationOnMock -> { + Runnable runnable = invocationOnMock.getArgument(0); + // run immediately + log.info("Running runnable immediately"); + runnable.run(); + return mock(ScheduledFuture.class); + }).when(eventLoop).schedule(any(Runnable.class), anyLong(), any()); topicListService = new TopicListService(pulsar, connection, true, 30); @@ -166,6 +184,33 @@ public void testCommandWatchSuccessResponse() { verify(connection.getCommandSender()).sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(topics), any()); } + @Test + public void testCommandWatchSuccessResponseWhenOutOfPermits() throws ExecutionException, InterruptedException { + // acquire all permits + AsyncDualMemoryLimiter.AsyncDualMemoryLimiterPermit permit = + memoryLimiter.acquire(1_000_000, AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, + Boolean.FALSE::booleanValue) + .get(); + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + List topics = Collections.singletonList("persistent://tenant/ns/topic1"); + String hash = TopicList.calculateHash(topics); + topicListFuture.complete(topics); + // wait for acquisition to timeout a few times + Thread.sleep(2000); + // release the permits + memoryLimiter.release(permit); + Awaitility.await().untilAsserted(() -> Assert.assertEquals(1, lookupSemaphore.availablePermits())); + verify(topicResources).registerPersistentTopicListener( + eq(NamespaceName.get("tenant/ns")), any(TopicListService.TopicListWatcher.class)); + verify(connection.getCommandSender()).sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(topics), any()); + } + @Test public void testCommandWatchErrorResponse() { topicListService.handleWatchTopicList( @@ -204,7 +249,7 @@ public void testCommandWatchTopicListCloseRemovesListener() { } @Test - public void testCommandWatchSuccessRetries() { + public void testCommandWatchSuccessDirectMemoryAcquirePermitsRetries() { topicListService.handleWatchTopicList( NamespaceName.get("tenant/ns"), 13, @@ -214,12 +259,6 @@ public void testCommandWatchSuccessRetries() { lookupSemaphore); List topics = Collections.singletonList("persistent://tenant/ns/topic1"); String hash = TopicList.calculateHash(topics); - doAnswer(invocationOnMock -> { - Runnable runnable = invocationOnMock.getArgument(0); - // run immediately - runnable.run(); - return mock(ScheduledFuture.class); - }).when(eventLoop).schedule(any(Runnable.class), anyLong(), any()); AtomicInteger failureCount = new AtomicInteger(0); doAnswer(invocationOnMock -> { if (failureCount.incrementAndGet() < 3) { @@ -279,12 +318,6 @@ public void testCommandWatchUpdateRetries() { List newTopics = Collections.singletonList("persistent://tenant/ns/topic2"); String hash = TopicList.calculateHash(ListUtils.union(topics, newTopics)); - doAnswer(invocationOnMock -> { - Runnable runnable = invocationOnMock.getArgument(0); - // run immediately - runnable.run(); - return mock(ScheduledFuture.class); - }).when(eventLoop).schedule(any(Runnable.class), anyLong(), any()); AtomicInteger failureCount = new AtomicInteger(0); doAnswer(invocationOnMock -> { if (failureCount.incrementAndGet() < 3) { From c3bfaded1e33f701c211797ef9602279235d6be4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Nov 2025 21:03:34 +0200 Subject: [PATCH 71/75] Remove possibly registered watcher future --- .../java/org/apache/pulsar/broker/service/TopicListService.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index f8eb5624e6f0e..f1d47b662c1a8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -352,6 +352,7 @@ public void initializeTopicsListWatcher(CompletableFuture watc + "watcherId={}.", connection, watcherId); watcher.close(); topicResources.deregisterPersistentTopicListener(watcher); + watchers.remove(watcherId, watcherFuture); } } }); From 71db086df6ee134bcc0ebbbab90129baeb26451c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Nov 2025 21:13:06 +0200 Subject: [PATCH 72/75] Remove unnecessary logic which removed previous watcher with same id - if there's a request for the same id, just use it. The client should use unique ids for different watchers --- .../broker/service/TopicListService.java | 28 ++----------------- 1 file changed, 3 insertions(+), 25 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index f1d47b662c1a8..540732805777c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -228,35 +228,13 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo CompletableFuture existingWatcherFuture = watchers.putIfAbsent(watcherId, watcherFuture); if (existingWatcherFuture != null) { - if (existingWatcherFuture.isDone() && !existingWatcherFuture.isCompletedExceptionally()) { - TopicListWatcher watcher = existingWatcherFuture.getNow(null); - log.info("[{}] Watcher with the same id is already created:" - + " watcherId={}, watcher={}", - connection.toString(), watcherId, watcher); - watcherFuture = existingWatcherFuture; - } else { - // There was an early request to create a watcher with the same watcherId. This can happen when - // client timeout is lower the broker timeouts. We need to wait until the previous watcher - // creation request either completes or fails. - log.warn("[{}] Watcher with id is already present on the connection," - + " consumerId={}", connection.toString(), watcherId); - ServerError error; - if (!existingWatcherFuture.isDone()) { - error = ServerError.ServiceNotReady; - } else { - error = ServerError.UnknownError; - watchers.remove(watcherId, existingWatcherFuture); - } - connection.getCommandSender().sendErrorResponse(requestId, error, - "Topic list watcher is already present on the connection"); - lookupSemaphore.release(); - return; - } + log.info("[{}] Watcher with the same watcherId={} is already created.", connection, watcherId); + // use the existing watcher if it's already created + watcherFuture = existingWatcherFuture; } else { initializeTopicsListWatcher(watcherFuture, namespaceName, watcherId, topicsPattern); } - CompletableFuture finalWatcherFuture = watcherFuture; finalWatcherFuture.thenAccept(watcher -> { List topicList = watcher.getMatchingTopics(); From 7c64540a8e2d806ba3d24915a5c8c70c97be5b34 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Nov 2025 11:10:02 +0200 Subject: [PATCH 73/75] Simplify TopicListSizeResultCache: use latest value --- .../TopicListSizeResultCache.java | 27 ++-------- .../TopicListSizeResultCacheTest.java | 50 +------------------ 2 files changed, 6 insertions(+), 71 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java index 9fe65abab93f2..90e3484479069 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCache.java @@ -21,7 +21,6 @@ import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; @@ -55,7 +54,6 @@ record CacheKey(String namespaceName, CommandGetTopicsOfNamespace.Mode mode) { public static class ResultHolder { private final AtomicReference> topicListSizeFuture = new AtomicReference<>(null); - private final AtomicLong existingSizeRef = new AtomicLong(-1L); /** * Get the topic list size estimate. The first request will return the initial estimate @@ -76,33 +74,18 @@ public CompletableFuture getSizeAsync() { } /** - * Update the topic list size estimate. The new estimated size will be updated by calculating the average - * of the existing and the new size. If the difference between the new and the existing size is less than 1, - * no update will be done. + * Update the topic list size estimate. The last changed value will be used. + * * @param actualSize the actual size of the topic list */ public void updateSize(long actualSize) { - long existingSizeValue = existingSizeRef.updateAndGet(existingSize -> { - if (existingSize > 0) { - // update by calculate the average actualSize of existing and the new actualSize - long updatedSize = (existingSize + actualSize) / 2; - // if the difference is more than 1, update the size - if (Math.abs(updatedSize - existingSize) > 1) { - return updatedSize; - } else { - return existingSize; - } - } else { - return actualSize; - } - }); CompletableFuture currentFuture = topicListSizeFuture.get(); if (currentFuture != null && !currentFuture.isDone()) { // complete the future if it's not done yet - currentFuture.complete(existingSizeValue); - } else if (currentFuture == null || currentFuture.getNow(0L).longValue() != existingSizeValue) { + currentFuture.complete(actualSize); + } else if (currentFuture == null || currentFuture.getNow(0L).longValue() != actualSize) { // only update the future if the current value is different from the existing value - topicListSizeFuture.compareAndSet(currentFuture, CompletableFuture.completedFuture(existingSizeValue)); + topicListSizeFuture.compareAndSet(currentFuture, CompletableFuture.completedFuture(actualSize)); } } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCacheTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCacheTest.java index 46beb96f02e8a..cea075886adbc 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCacheTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/topiclistlimit/TopicListSizeResultCacheTest.java @@ -164,54 +164,6 @@ public void testResultHolder_updateSize_firstUpdate() { assertEquals(nextFuture.join().longValue(), actualSize); } - @Test - public void testResultHolder_updateSize_calculatesAverage() { - TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); - - // First update - holder.updateSize(10000L); - CompletableFuture future1 = holder.getSizeAsync(); - assertEquals(future1.join().longValue(), 10000L); - - // Second update - should average - holder.updateSize(20000L); - CompletableFuture future2 = holder.getSizeAsync(); - assertEquals(future2.join().longValue(), 15000L, "Should calculate average: (10000 + 20000) / 2"); - - // Third update - should average again - holder.updateSize(30000L); - CompletableFuture future3 = holder.getSizeAsync(); - assertEquals(future3.join().longValue(), 22500L, "Should calculate average: (15000 + 30000) / 2"); - } - - @Test - public void testResultHolder_updateSize_noUpdateWhenDifferenceIsSmall() { - TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); - - // First update - holder.updateSize(1000L); - assertEquals(holder.getSizeAsync().join().longValue(), 1000L); - - // Second update with small difference (average would be 1001) - holder.updateSize(1002L); - // Should not update because abs(1001 - 1000) <= 1 - assertEquals(holder.getSizeAsync().join().longValue(), 1000L, "Should not update when difference <= 1"); - } - - @Test - public void testResultHolder_updateSize_updatesWhenDifferenceIsLarge() { - TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); - - // First update - holder.updateSize(1000L); - assertEquals(holder.getSizeAsync().join().longValue(), 1000L); - - // Second update with large difference (average would be 1500) - holder.updateSize(2000L); - // Should update because abs(1500 - 1000) > 1 - assertEquals(holder.getSizeAsync().join().longValue(), 1500L, "Should update when difference > 1"); - } - @Test public void testResultHolder_updateSize_completesWaitingFuture() throws Exception { TopicListSizeResultCache.ResultHolder holder = new TopicListSizeResultCache.ResultHolder(); @@ -296,7 +248,7 @@ public void testResultHolder_multipleSequentialRequests() throws Exception { // Third request CompletableFuture future3 = holder.getSizeAsync(); - assertEquals(future3.get().longValue(), 20000L, "Should be average of 15000 and 25000"); + assertEquals(future3.get().longValue(), 25000L, "Should be the last value"); } @Test From 896a8d256391376c41576dd2dd146caf87b1602c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Nov 2025 11:39:36 +0200 Subject: [PATCH 74/75] Fix typo --- .../api/PatternConsumerBackPressureMultipleConsumersTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java index e012a3075b61d..d91106ab68342 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerBackPressureMultipleConsumersTest.java @@ -103,7 +103,7 @@ public void testGetTopicsWithLargeAmountOfConcurrentClientConnections() @Cleanup PulsarClientSharedResources sharedResources = PulsarClientSharedResources.builder().build(); - List clients = new ArrayList<>(requests); + List clients = new ArrayList<>(numberOfClients); @Cleanup Closeable closeClients = () -> { for (PulsarClient client : clients) { From 09cadc2c697879df74e8458129c65880972e368c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Nov 2025 16:28:20 +0200 Subject: [PATCH 75/75] Revert changes to topic list watcher --- .../broker/service/PulsarCommandSender.java | 9 +- .../service/PulsarCommandSenderImpl.java | 23 +- .../broker/service/TopicListService.java | 286 ++++-------------- .../broker/service/TopicListServiceTest.java | 215 +------------ .../broker/service/TopicListWatcherTest.java | 14 +- 5 files changed, 78 insertions(+), 469 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java index 7b144340c3191..8e2deff31d0b5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSender.java @@ -96,11 +96,8 @@ Future sendMessagesToConsumer(long consumerId, String topicName, Subscript void sendEndTxnErrorResponse(long requestId, TxnID txnID, ServerError error, String message); - CompletableFuture sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, - List topics, - Consumer permitAcquireErrorHandler); + void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics); - CompletableFuture sendWatchTopicListUpdate(long watcherId, - List newTopics, List deletedTopics, String topicsHash, - Consumer permitAcquireErrorHandler); + void sendWatchTopicListUpdate(long watcherId, + List newTopics, List deletedTopics, String topicsHash); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java index 32b699f69cdb7..275e049255c35 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java @@ -366,30 +366,27 @@ public void sendEndTxnErrorResponse(long requestId, TxnID txnID, ServerError err /*** * @param topics topic names which are matching, the topic name contains the partition suffix. - * @return */ @Override - public CompletableFuture sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, - List topics, - Consumer permitAcquireErrorHandler) { + public void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics) { BaseCommand command = Commands.newWatchTopicListSuccess(requestId, watcherId, topicsHash, topics); - safeIntercept(command, cnx); - return acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), - command, permitAcquireErrorHandler); + interceptAndWriteCommand(command); } /*** * {@inheritDoc} - * @return */ @Override - public CompletableFuture sendWatchTopicListUpdate(long watcherId, List newTopics, - List deletedTopics, String topicsHash, - Consumer permitAcquireErrorHandler) { + public void sendWatchTopicListUpdate(long watcherId, + List newTopics, List deletedTopics, String topicsHash) { BaseCommand command = Commands.newWatchTopicUpdate(watcherId, newTopics, deletedTopics, topicsHash); + interceptAndWriteCommand(command); + } + + private void interceptAndWriteCommand(BaseCommand command) { safeIntercept(command, cnx); - return acquireDirectMemoryPermitsAndWriteAndFlush(cnx.ctx(), maxTopicListInFlightLimiter, () -> !cnx.isActive(), - command, permitAcquireErrorHandler); + ByteBuf outBuf = Commands.serializeWithSize(command); + writeAndFlush(outBuf); } private void writeAndFlush(ByteBuf outBuf) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index 540732805777c..ef2ea284cf783 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -18,48 +18,30 @@ */ package org.apache.pulsar.broker.service; -import com.google.common.annotations.VisibleForTesting; import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.concurrent.BlockingDeque; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executor; -import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; -import java.util.function.BooleanSupplier; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.function.Supplier; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.TopicResources; -import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; -import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; -import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.api.proto.CommandWatchTopicListClose; import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; -import org.apache.pulsar.common.semaphore.AsyncSemaphore; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.topics.TopicsPattern; import org.apache.pulsar.common.topics.TopicsPatternFactory; -import org.apache.pulsar.common.util.Backoff; -import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.Runnables; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; import org.apache.pulsar.metadata.api.NotificationType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TopicListService { + + public static class TopicListWatcher implements BiConsumer { /** Topic names which are matching, the topic name contains the partition suffix. **/ @@ -68,21 +50,15 @@ public static class TopicListWatcher implements BiConsumer sendTopicListUpdateTasksBeforeInit = new LinkedBlockingDeque<>(); /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). */ public TopicListWatcher(TopicListService topicListService, long id, - TopicsPattern topicsPattern, List topics, - Executor executor) { + TopicsPattern topicsPattern, List topics) { this.topicListService = topicListService; this.id = id; this.topicsPattern = topicsPattern; - this.executor = executor; this.matchingTopics = TopicList.filterTopics(topics, topicsPattern); } @@ -95,9 +71,6 @@ public List getMatchingTopics() { */ @Override public void accept(String topicName, NotificationType notificationType) { - if (closed) { - return; - } String partitionedTopicName = TopicName.get(topicName).getPartitionedTopicName(); String domainLessTopicName = TopicList.removeTopicDomainScheme(partitionedTopicName); @@ -114,41 +87,9 @@ public void accept(String topicName, NotificationType notificationType) { matchingTopics.add(topicName); } String hash = TopicList.calculateHash(matchingTopics); - sendTopicListUpdate(hash, deletedTopics, newTopics); + topicListService.sendTopicListUpdate(id, hash, deletedTopics, newTopics); } } - - private synchronized void sendTopicListUpdate(String hash, List deletedTopics, List newTopics) { - if (closed) { - return; - } - Runnable task = () -> topicListService.sendTopicListUpdate(id, hash, deletedTopics, newTopics); - if (sendTopicListSuccessCompleted) { - executor.execute(task); - } else { - // if sendTopicListSuccess hasn't completed, add to a queue to be executed after it completes - sendTopicListUpdateTasksBeforeInit.add(task); - } - } - - @VisibleForTesting - synchronized void sendTopicListSuccessCompleted() { - if (closed) { - sendTopicListUpdateTasksBeforeInit.clear(); - return; - } - // Drain all pending sendTopicListUpdate tasks - Runnable task; - while ((task = sendTopicListUpdateTasksBeforeInit.poll()) != null) { - executor.execute(task); - } - sendTopicListSuccessCompleted = true; - } - - public synchronized void close() { - closed = true; - sendTopicListUpdateTasksBeforeInit.clear(); - } } @@ -156,18 +97,15 @@ public synchronized void close() { private final NamespaceService namespaceService; private final TopicResources topicResources; - private final PulsarService pulsar; private final ServerCnx connection; private final boolean enableSubscriptionPatternEvaluation; private final int maxSubscriptionPatternLength; private final ConcurrentLongHashMap> watchers; - private final Backoff retryBackoff; public TopicListService(PulsarService pulsar, ServerCnx connection, boolean enableSubscriptionPatternEvaluation, int maxSubscriptionPatternLength) { this.namespaceService = pulsar.getNamespaceService(); - this.pulsar = pulsar; this.connection = connection; this.enableSubscriptionPatternEvaluation = enableSubscriptionPatternEvaluation; this.maxSubscriptionPatternLength = maxSubscriptionPatternLength; @@ -176,10 +114,6 @@ public TopicListService(PulsarService pulsar, ServerCnx connection, .concurrencyLevel(1) .build(); this.topicResources = pulsar.getPulsarResources().getTopicResources(); - this.retryBackoff = new Backoff( - 100, TimeUnit.MILLISECONDS, - 25, TimeUnit.SECONDS, - 0, TimeUnit.MILLISECONDS); } public void inactivate() { @@ -228,13 +162,35 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo CompletableFuture existingWatcherFuture = watchers.putIfAbsent(watcherId, watcherFuture); if (existingWatcherFuture != null) { - log.info("[{}] Watcher with the same watcherId={} is already created.", connection, watcherId); - // use the existing watcher if it's already created - watcherFuture = existingWatcherFuture; + if (existingWatcherFuture.isDone() && !existingWatcherFuture.isCompletedExceptionally()) { + TopicListWatcher watcher = existingWatcherFuture.getNow(null); + log.info("[{}] Watcher with the same id is already created:" + + " watcherId={}, watcher={}", + connection.toString(), watcherId, watcher); + watcherFuture = existingWatcherFuture; + } else { + // There was an early request to create a watcher with the same watcherId. This can happen when + // client timeout is lower the broker timeouts. We need to wait until the previous watcher + // creation request either completes or fails. + log.warn("[{}] Watcher with id is already present on the connection," + + " consumerId={}", connection.toString(), watcherId); + ServerError error; + if (!existingWatcherFuture.isDone()) { + error = ServerError.ServiceNotReady; + } else { + error = ServerError.UnknownError; + watchers.remove(watcherId, existingWatcherFuture); + } + connection.getCommandSender().sendErrorResponse(requestId, error, + "Topic list watcher is already present on the connection"); + lookupSemaphore.release(); + return; + } } else { initializeTopicsListWatcher(watcherFuture, namespaceName, watcherId, topicsPattern); } + CompletableFuture finalWatcherFuture = watcherFuture; finalWatcherFuture.thenAccept(watcher -> { List topicList = watcher.getMatchingTopics(); @@ -247,8 +203,7 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo "[{}] Received WatchTopicList for namespace [//{}] by {}", connection.toString(), namespaceName, requestId); } - sendTopicListSuccessWithPermitAcquiringRetries(watcherId, requestId, topicList, hash, - watcher::sendTopicListSuccessCompleted); + connection.getCommandSender().sendWatchTopicListSuccess(requestId, watcherId, hash, topicList); lookupSemaphore.release(); }) .exceptionally(ex -> { @@ -263,100 +218,31 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo }); } - private void sendTopicListSuccessWithPermitAcquiringRetries(long watcherId, long requestId, List topicList, - String hash, - Runnable successfulCompletionCallback) { - performOperationWithPermitAcquiringRetries(watcherId, "topic list success", permitAcquireErrorHandler -> - () -> connection.getCommandSender() - .sendWatchTopicListSuccess(requestId, watcherId, hash, topicList, permitAcquireErrorHandler) - .whenComplete((__, t) -> { - if (t != null) { - // this is an unexpected case - log.warn("[{}] Failed to send topic list success for watcherId={}. Watcher will be in " - + "inconsistent state.", connection, watcherId, t); - } else { - // completed successfully, run the callback - successfulCompletionCallback.run(); - } - })); - } - /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). */ public void initializeTopicsListWatcher(CompletableFuture watcherFuture, NamespaceName namespace, long watcherId, TopicsPattern topicsPattern) { - BooleanSupplier isPermitRequestCancelled = () -> !connection.isActive() || !watchers.containsKey(watcherId); - if (isPermitRequestCancelled.getAsBoolean()) { - return; - } - TopicListSizeResultCache.ResultHolder listSizeHolder = pulsar.getBrokerService().getTopicListSizeResultCache() - .getTopicListSize(namespace.toString(), CommandGetTopicsOfNamespace.Mode.PERSISTENT); - AsyncDualMemoryLimiter maxTopicListInFlightLimiter = pulsar.getBrokerService().getMaxTopicListInFlightLimiter(); - - listSizeHolder.getSizeAsync().thenCompose(initialSize -> { - // use heap size limiter to avoid broker getting overwhelmed by a lot of concurrent topic list requests - return maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, - AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { - AtomicReference watcherRef = new AtomicReference<>(); - return namespaceService.getListOfPersistentTopics(namespace).thenCompose(topics -> { - long actualSize = TopicListMemoryLimiter.estimateTopicListSize(topics); - listSizeHolder.updateSize(actualSize); - // register watcher immediately so that we don't lose events - TopicListWatcher watcher = - new TopicListWatcher(this, watcherId, topicsPattern, topics, - connection.ctx().executor()); - watcherRef.set(watcher); - topicResources.registerPersistentTopicListener(namespace, watcher); - // use updated permits to slow down responses so that backpressure gets applied - return maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, - isPermitRequestCancelled, updatedPermits -> { - // reset retry backoff - retryBackoff.reset(); - // just return the watcher which was already created before - return CompletableFuture.completedFuture(watcher); - }, CompletableFuture::failedFuture); - }).whenComplete((watcher, exception) -> { - if (exception != null) { - if (watcherRef.get() != null) { - watcher.close(); - topicResources.deregisterPersistentTopicListener(watcherRef.get()); - } - // triggers a retry - throw FutureUtil.wrapToCompletionException(exception); - } else { - if (!watcherFuture.complete(watcher)) { - log.warn("[{}] Watcher future was already completed. Deregistering " - + "watcherId={}.", connection, watcherId); - watcher.close(); - topicResources.deregisterPersistentTopicListener(watcher); - watchers.remove(watcherId, watcherFuture); - } - } - }); - }, CompletableFuture::failedFuture); - }).exceptionally(t -> { - Throwable unwrappedException = FutureUtil.unwrapCompletionException(t); - if (!isPermitRequestCancelled.getAsBoolean() && ( - unwrappedException instanceof AsyncSemaphore.PermitAcquireTimeoutException - || unwrappedException instanceof AsyncSemaphore.PermitAcquireQueueFullException)) { - // retry with backoff if permit acquisition fails due to timeout or queue full - long retryAfterMillis = this.retryBackoff.next(); - log.info("[{}] {} when initializing topic list watcher watcherId={} for namespace {}. Retrying in {} " - + "ms.", connection, unwrappedException.getMessage(), watcherId, namespace, - retryAfterMillis); - connection.ctx().executor() - .schedule(() -> initializeTopicsListWatcher(watcherFuture, namespace, watcherId, topicsPattern), - retryAfterMillis, TimeUnit.MILLISECONDS); - } else { - log.warn("[{}] Failed to initialize topic list watcher watcherId={} for namespace {}.", connection, - watcherId, namespace, unwrappedException); - watcherFuture.completeExceptionally(unwrappedException); - } - return null; - }); + namespaceService.getListOfPersistentTopics(namespace). + thenApply(topics -> { + TopicListWatcher watcher = new TopicListWatcher(this, watcherId, topicsPattern, topics); + topicResources.registerPersistentTopicListener(namespace, watcher); + return watcher; + }). + whenComplete((watcher, exception) -> { + if (exception != null) { + watcherFuture.completeExceptionally(exception); + } else { + if (!watcherFuture.complete(watcher)) { + log.warn("[{}] Watcher future was already completed. Deregistering watcherId={}.", + connection.toString(), watcherId); + topicResources.deregisterPersistentTopicListener(watcher); + } + } + }); } + public void handleWatchTopicListClose(CommandWatchTopicListClose commandWatchTopicListClose) { long requestId = commandWatchTopicListClose.getRequestId(); long watcherId = commandWatchTopicListClose.getWatcherId(); @@ -365,7 +251,7 @@ public void handleWatchTopicListClose(CommandWatchTopicListClose commandWatchTop } public void deleteTopicListWatcher(Long watcherId) { - CompletableFuture watcherFuture = watchers.remove(watcherId); + CompletableFuture watcherFuture = watchers.get(watcherId); if (watcherFuture == null) { log.info("[{}] TopicListWatcher was not registered on the connection: {}", watcherId, connection.toString()); @@ -379,23 +265,20 @@ public void deleteTopicListWatcher(Long watcherId) { // create operation will complete, the new watcher will be discarded. log.info("[{}] Closed watcher before its creation was completed. watcherId={}", connection.toString(), watcherId); + watchers.remove(watcherId); return; } - // deregister topic listener while avoiding race conditions - watcherFuture.whenComplete((watcher, t) -> { - if (watcher != null) { - topicResources.deregisterPersistentTopicListener(watcher); - watcher.close(); - } - }); - if (watcherFuture.isCompletedExceptionally()) { log.info("[{}] Closed watcher that already failed to be created. watcherId={}", connection.toString(), watcherId); + watchers.remove(watcherId); return; } + // Proceed with normal watcher close + topicResources.deregisterPersistentTopicListener(watcherFuture.getNow(null)); + watchers.remove(watcherId); log.info("[{}] Closed watcher, watcherId={}", connection.toString(), watcherId); } @@ -405,67 +288,8 @@ public void deleteTopicListWatcher(Long watcherId) { */ public void sendTopicListUpdate(long watcherId, String topicsHash, List deletedTopics, List newTopics) { - performOperationWithPermitAcquiringRetries(watcherId, "topic list update", permitAcquireErrorHandler -> - () -> connection.getCommandSender() - .sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash, - permitAcquireErrorHandler) - .whenComplete((__, t) -> { - if (t != null) { - // this is an unexpected case - log.warn("[{}] Failed to send topic list update for watcherId={}. Watcher will be in " - + "inconsistent state.", connection, watcherId, t); - } - })); - } - - // performs an operation with infinite permit acquiring retries. - // If acquiring permits fails, it will retry after a backoff period - private void performOperationWithPermitAcquiringRetries(long watcherId, String operationName, - Function, - Supplier>> - asyncOperationFactory) { - // holds a reference to the operation, this is to resolve a circular dependency between the error handler and - // the actual operation - AtomicReference operationRef = new AtomicReference<>(); - // create the error handler for the operation - Consumer permitAcquireErrorHandler = - createPermitAcquireErrorHandler(watcherId, operationName, operationRef); - // create the async operation using the factory function. Pass the error handler to the factory function. - Supplier> asyncOperation = asyncOperationFactory.apply(permitAcquireErrorHandler); - // set the operation to run into the operation reference - operationRef.set(Runnables.catchingAndLoggingThrowables(() -> { - if (!connection.isActive() || !watchers.containsKey(watcherId)) { - // do nothing if the connection has already been closed or the watcher has been removed - return; - } - asyncOperation.get().thenRun(() -> retryBackoff.reset()); - })); - // run the operation - operationRef.get().run(); + connection.getCommandSender().sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash); } - // retries acquiring permits until the connection is closed or the watcher is removed - private Consumer createPermitAcquireErrorHandler(long watcherId, String operationName, - AtomicReference operationRef) { - ScheduledExecutorService scheduledExecutor = connection.ctx().channel().eventLoop(); - AtomicInteger retryCount = new AtomicInteger(0); - return t -> { - Throwable unwrappedException = FutureUtil.unwrapCompletionException(t); - if (unwrappedException instanceof AsyncSemaphore.PermitAcquireCancelledException - || unwrappedException instanceof AsyncSemaphore.PermitAcquireAlreadyClosedException - || !connection.isActive() - || !watchers.containsKey(watcherId)) { - return; - } - long retryDelay = retryBackoff.next(); - log.info("[{}] Cannot acquire direct memory tokens for sending {}. Retry {} in {} ms. {}", connection, - operationName, retryCount.get(), retryDelay, t.getMessage()); - scheduledExecutor.schedule(operationRef.get(), retryDelay, TimeUnit.MILLISECONDS); - }; - } - @VisibleForTesting - CompletableFuture getWatcherFuture(long watcherId) { - return watchers.get(watcherId); - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java index 6c8cebd0016d6..9109828c025b6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java @@ -18,62 +18,31 @@ */ package org.apache.pulsar.broker.service; -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.timeout; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; -import io.netty.channel.Channel; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.EventLoop; -import io.netty.util.concurrent.EventExecutor; -import io.netty.util.concurrent.ImmediateEventExecutor; -import io.netty.util.concurrent.ScheduledFuture; import java.net.InetSocketAddress; -import java.time.Duration; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Semaphore; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.collections4.ListUtils; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.resources.TopicResources; -import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; import org.apache.pulsar.common.api.proto.CommandWatchTopicListClose; import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.naming.NamespaceName; -import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; -import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; -import org.apache.pulsar.common.semaphore.AsyncSemaphore; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.topics.TopicsPattern; -import org.apache.pulsar.metadata.api.MetadataStore; -import org.apache.pulsar.metadata.api.Notification; -import org.apache.pulsar.metadata.api.NotificationType; -import org.awaitility.Awaitility; import org.testng.Assert; -import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -@Slf4j public class TopicListServiceTest { private TopicListService topicListService; @@ -83,25 +52,13 @@ public class TopicListServiceTest { private TopicResources topicResources; private final TopicsPattern.RegexImplementation topicsPatternImplementation = TopicsPattern.RegexImplementation.RE2J_WITH_JDK_FALLBACK; - private EventLoop eventLoop; - private PulsarCommandSender pulsarCommandSender; - private Consumer notificationConsumer; - private AsyncDualMemoryLimiterImpl memoryLimiter; @BeforeMethod(alwaysRun = true) public void setup() throws Exception { lookupSemaphore = new Semaphore(1); lookupSemaphore.acquire(); topicListFuture = new CompletableFuture<>(); - - AtomicReference> listenerRef = new AtomicReference<>(); - MetadataStore metadataStore = mock(MetadataStore.class); - doAnswer(invocationOnMock -> { - listenerRef.set(invocationOnMock.getArgument(0)); - return null; - }).when(metadataStore).registerListener(any()); - topicResources = spy(new TopicResources(metadataStore)); - notificationConsumer = listenerRef.get(); + topicResources = mock(TopicResources.class); PulsarService pulsar = mock(PulsarService.class); when(pulsar.getNamespaceService()).thenReturn(mock(NamespaceService.class)); @@ -109,62 +66,15 @@ public void setup() throws Exception { when(pulsar.getPulsarResources().getTopicResources()).thenReturn(topicResources); when(pulsar.getNamespaceService().getListOfPersistentTopics(any())).thenReturn(topicListFuture); - BrokerService brokerService = mock(BrokerService.class); - when(pulsar.getBrokerService()).thenReturn(brokerService); - TopicListSizeResultCache topicListSizeResultCache = mock(TopicListSizeResultCache.class); - when(brokerService.getTopicListSizeResultCache()).thenReturn(topicListSizeResultCache); - TopicListSizeResultCache.ResultHolder resultHolder = mock(TopicListSizeResultCache.ResultHolder.class); - doReturn(resultHolder).when(topicListSizeResultCache).getTopicListSize(anyString(), any()); - doReturn(CompletableFuture.completedFuture(1L)).when(resultHolder).getSizeAsync(); - - memoryLimiter = new AsyncDualMemoryLimiterImpl(1_000_000, 10000, 500, 1_000_000, 10000, 500); - doReturn(memoryLimiter).when(brokerService).getMaxTopicListInFlightLimiter(); connection = mock(ServerCnx.class); when(connection.getRemoteAddress()).thenReturn(new InetSocketAddress(10000)); - pulsarCommandSender = mock(PulsarCommandSender.class); - when(connection.getCommandSender()).thenReturn(pulsarCommandSender); - when(connection.isActive()).thenReturn(true); - when(pulsarCommandSender.sendWatchTopicListUpdate(anyLong(), any(), any(), anyString(), any())) - .thenReturn(CompletableFuture.completedFuture(null)); - when(pulsarCommandSender.sendWatchTopicListSuccess(anyLong(), anyLong(), anyString(), any(), any())) - .thenReturn(CompletableFuture.completedFuture(null)); - - - ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); - when(connection.ctx()).thenReturn(ctx); - EventExecutor executor = spy(ImmediateEventExecutor.INSTANCE); - doReturn(executor).when(ctx).executor(); - doAnswer(invocationOnMock -> { - Runnable runnable = invocationOnMock.getArgument(0); - // run immediately - log.info("Running runnable immediately"); - runnable.run(); - return mock(ScheduledFuture.class); - }).when(executor).schedule(any(Runnable.class), anyLong(), any()); - Channel channel = mock(Channel.class); - when(ctx.channel()).thenReturn(channel); - eventLoop = mock(EventLoop.class); - when(channel.eventLoop()).thenReturn(eventLoop); - doAnswer(invocationOnMock -> { - Runnable runnable = invocationOnMock.getArgument(0); - // run immediately - log.info("Running runnable immediately"); - runnable.run(); - return mock(ScheduledFuture.class); - }).when(eventLoop).schedule(any(Runnable.class), anyLong(), any()); + when(connection.getCommandSender()).thenReturn(mock(PulsarCommandSender.class)); topicListService = new TopicListService(pulsar, connection, true, 30); } - @AfterMethod(alwaysRun = true) - void cleanup() { - if (memoryLimiter != null) { - memoryLimiter.close(); - } - } - @Test public void testCommandWatchSuccessResponse() { @@ -178,37 +88,10 @@ public void testCommandWatchSuccessResponse() { List topics = Collections.singletonList("persistent://tenant/ns/topic1"); String hash = TopicList.calculateHash(topics); topicListFuture.complete(topics); - Awaitility.await().untilAsserted(() -> Assert.assertEquals(1, lookupSemaphore.availablePermits())); + Assert.assertEquals(1, lookupSemaphore.availablePermits()); verify(topicResources).registerPersistentTopicListener( eq(NamespaceName.get("tenant/ns")), any(TopicListService.TopicListWatcher.class)); - verify(connection.getCommandSender()).sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(topics), any()); - } - - @Test - public void testCommandWatchSuccessResponseWhenOutOfPermits() throws ExecutionException, InterruptedException { - // acquire all permits - AsyncDualMemoryLimiter.AsyncDualMemoryLimiterPermit permit = - memoryLimiter.acquire(1_000_000, AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, - Boolean.FALSE::booleanValue) - .get(); - topicListService.handleWatchTopicList( - NamespaceName.get("tenant/ns"), - 13, - 7, - "persistent://tenant/ns/topic\\d", - topicsPatternImplementation, null, - lookupSemaphore); - List topics = Collections.singletonList("persistent://tenant/ns/topic1"); - String hash = TopicList.calculateHash(topics); - topicListFuture.complete(topics); - // wait for acquisition to timeout a few times - Thread.sleep(2000); - // release the permits - memoryLimiter.release(permit); - Awaitility.await().untilAsserted(() -> Assert.assertEquals(1, lookupSemaphore.availablePermits())); - verify(topicResources).registerPersistentTopicListener( - eq(NamespaceName.get("tenant/ns")), any(TopicListService.TopicListWatcher.class)); - verify(connection.getCommandSender()).sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(topics), any()); + verify(connection.getCommandSender()).sendWatchTopicListSuccess(7, 13, hash, topics); } @Test @@ -221,7 +104,7 @@ public void testCommandWatchErrorResponse() { topicsPatternImplementation, null, lookupSemaphore); topicListFuture.completeExceptionally(new PulsarServerException("Error")); - Awaitility.await().untilAsserted(() -> Assert.assertEquals(1, lookupSemaphore.availablePermits())); + Assert.assertEquals(1, lookupSemaphore.availablePermits()); verifyNoInteractions(topicResources); verify(connection.getCommandSender()).sendErrorResponse(eq(7L), any(ServerError.class), eq(PulsarServerException.class.getCanonicalName() + ": Error")); @@ -238,100 +121,12 @@ public void testCommandWatchTopicListCloseRemovesListener() { lookupSemaphore); List topics = Collections.singletonList("persistent://tenant/ns/topic1"); topicListFuture.complete(topics); - assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(1)); CommandWatchTopicListClose watchTopicListClose = new CommandWatchTopicListClose() .setRequestId(8) .setWatcherId(13); topicListService.handleWatchTopicListClose(watchTopicListClose); - verify(topicResources).deregisterPersistentTopicListener(any(TopicListService.TopicListWatcher.class)); } - @Test - public void testCommandWatchSuccessDirectMemoryAcquirePermitsRetries() { - topicListService.handleWatchTopicList( - NamespaceName.get("tenant/ns"), - 13, - 7, - "persistent://tenant/ns/topic\\d", - topicsPatternImplementation, null, - lookupSemaphore); - List topics = Collections.singletonList("persistent://tenant/ns/topic1"); - String hash = TopicList.calculateHash(topics); - AtomicInteger failureCount = new AtomicInteger(0); - doAnswer(invocationOnMock -> { - if (failureCount.incrementAndGet() < 3) { - Throwable failure = new AsyncSemaphore.PermitAcquireTimeoutException("Acquire timed out"); - Consumer permitAcquireErrorHandler = invocationOnMock.getArgument(4); - permitAcquireErrorHandler.accept(failure); - return CompletableFuture.failedFuture(failure); - } else { - return CompletableFuture.completedFuture(null); - } - }).when(pulsarCommandSender).sendWatchTopicListSuccess(anyLong(), anyLong(), anyString(), any(), any()); - topicListFuture.complete(topics); - assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(1)); - verify(connection.getCommandSender(), times(3)) - .sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(topics), any()); - } - - @Test - public void testCommandWatchUpdate() { - topicListService.handleWatchTopicList( - NamespaceName.get("tenant/ns"), - 13, - 7, - "persistent://tenant/ns/topic\\d", - topicsPatternImplementation, null, - lookupSemaphore); - List topics = Collections.singletonList("persistent://tenant/ns/topic1"); - topicListFuture.complete(topics); - assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(1)); - - List newTopics = Collections.singletonList("persistent://tenant/ns/topic2"); - String hash = TopicList.calculateHash(ListUtils.union(topics, newTopics)); - notificationConsumer.accept( - new Notification(NotificationType.Created, "/managed-ledgers/tenant/ns/persistent/topic2")); - verify(connection.getCommandSender(), timeout(1000L)) - .sendWatchTopicListUpdate(eq(13L), eq(newTopics), any(), eq(hash), any()); - - hash = TopicList.calculateHash(newTopics); - notificationConsumer.accept( - new Notification(NotificationType.Deleted, "/managed-ledgers/tenant/ns/persistent/topic1")); - verify(connection.getCommandSender(), timeout(1000L)) - .sendWatchTopicListUpdate(eq(13L), eq(List.of()), eq(topics), eq(hash), any()); - } - - @Test - public void testCommandWatchUpdateRetries() { - topicListService.handleWatchTopicList( - NamespaceName.get("tenant/ns"), - 13, - 7, - "persistent://tenant/ns/topic\\d", - topicsPatternImplementation, null, - lookupSemaphore); - List topics = Collections.singletonList("persistent://tenant/ns/topic1"); - topicListFuture.complete(topics); - assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(1)); - - List newTopics = Collections.singletonList("persistent://tenant/ns/topic2"); - String hash = TopicList.calculateHash(ListUtils.union(topics, newTopics)); - AtomicInteger failureCount = new AtomicInteger(0); - doAnswer(invocationOnMock -> { - if (failureCount.incrementAndGet() < 3) { - Throwable failure = new AsyncSemaphore.PermitAcquireTimeoutException("Acquire timed out"); - Consumer permitAcquireErrorHandler = invocationOnMock.getArgument(4); - permitAcquireErrorHandler.accept(failure); - return CompletableFuture.failedFuture(failure); - } else { - return CompletableFuture.completedFuture(null); - } - }).when(pulsarCommandSender).sendWatchTopicListUpdate(anyLong(), any(), any(), anyString(), any()); - notificationConsumer.accept( - new Notification(NotificationType.Created, "/managed-ledgers/tenant/ns/persistent/topic2")); - verify(connection.getCommandSender(), timeout(1000L).times(3)) - .sendWatchTopicListUpdate(eq(13L), eq(newTopics), eq(List.of()), eq(hash), any()); - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java index 92dddd785a5f8..884cdc0ef9266 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java @@ -18,11 +18,9 @@ */ package org.apache.pulsar.broker.service; -import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; -import com.google.common.util.concurrent.MoreExecutors; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -55,9 +53,7 @@ public class TopicListWatcherTest { @BeforeMethod(alwaysRun = true) public void setup() { topicListService = mock(TopicListService.class); - watcher = new TopicListService.TopicListWatcher(topicListService, ID, PATTERN, INITIAL_TOPIC_LIST, - MoreExecutors.directExecutor()); - watcher.sendTopicListSuccessCompleted(); + watcher = new TopicListService.TopicListWatcher(topicListService, ID, PATTERN, INITIAL_TOPIC_LIST); } @Test @@ -75,8 +71,8 @@ public void testAcceptSendsNotificationAndRemembersTopic() { List allMatchingTopics = Arrays.asList( "persistent://tenant/ns/topic1", "persistent://tenant/ns/topic2", newTopic); String hash = TopicList.calculateHash(allMatchingTopics); - verify(topicListService).sendTopicListUpdate(eq(ID), eq(hash), eq(Collections.emptyList()), - eq(Collections.singletonList(newTopic))); + verify(topicListService).sendTopicListUpdate(ID, hash, Collections.emptyList(), + Collections.singletonList(newTopic)); Assert.assertEquals( allMatchingTopics, watcher.getMatchingTopics()); @@ -89,8 +85,8 @@ public void testAcceptSendsNotificationAndForgetsTopic() { List allMatchingTopics = Collections.singletonList("persistent://tenant/ns/topic2"); String hash = TopicList.calculateHash(allMatchingTopics); - verify(topicListService).sendTopicListUpdate(eq(ID), eq(hash), - eq(Collections.singletonList(deletedTopic)), eq(Collections.emptyList())); + verify(topicListService).sendTopicListUpdate(ID, hash, + Collections.singletonList(deletedTopic), Collections.emptyList()); Assert.assertEquals( allMatchingTopics, watcher.getMatchingTopics());